diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptReport.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptReport.java
index 031573f..53c18ae 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptReport.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptReport.java
@@ -51,14 +51,15 @@
@Unstable
public static ApplicationAttemptReport newInstance(
ApplicationAttemptId applicationAttemptId, String host, int rpcPort,
- String url, String diagnostics, YarnApplicationAttemptState state,
- ContainerId amContainerId) {
+ String url, String oUrl, String diagnostics,
+ YarnApplicationAttemptState state, ContainerId amContainerId) {
ApplicationAttemptReport report =
Records.newRecord(ApplicationAttemptReport.class);
report.setApplicationAttemptId(applicationAttemptId);
report.setHost(host);
report.setRpcPort(rpcPort);
report.setTrackingUrl(url);
+ report.setOriginalTrackingUrl(oUrl);
report.setDiagnostics(diagnostics);
report.setYarnApplicationAttemptState(state);
report.setAMContainerId(amContainerId);
@@ -136,6 +137,19 @@ public abstract void setYarnApplicationAttemptState(
public abstract void setTrackingUrl(String url);
/**
+ * Get the original tracking url for the application attempt.
+ *
+ * @return original tracking url for the application attempt
+ */
+ @Public
+ @Unstable
+ public abstract String getOriginalTrackingUrl();
+
+ @Private
+ @Unstable
+ public abstract void setOriginalTrackingUrl(String oUrl);
+
+ /**
* Get the ApplicationAttemptId of this attempt of the
* application
*
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 59e108a..d54e9cb 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -1203,6 +1203,19 @@
public static final String TIMELINE_SERVICE_KEYTAB =
TIMELINE_SERVICE_PREFIX + "keytab";
+ /**
+ * The setting that controls whether yarn metrics is published on the
+ * timeline server or not.
+ */
+ public static final String YARN_METRICS_PUBLISHER_ENABLED =
+ YARN_PREFIX + "metrics-publisher.enabled";
+ public static final boolean DEFAULT_YARN_METRICS_PUBLISHER_ENABLED = false;
+
+ public static final String YARN_METRICS_PUBLISHER_MULTI_THREADED_DISPATCHER_POOL_SIZE =
+ RM_PREFIX + "metrics-publisher.multi-threaded-dispatcher.pool-size";
+ public static final int DEFAULT_YARN_METRICS_PUBLISHER_MULTI_THREADED_DISPATCHER_POOL_SIZE =
+ 10;
+
////////////////////////////////
// Other Configs
////////////////////////////////
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index 3f1fa6c..897df12 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -197,9 +197,10 @@ message ApplicationAttemptReportProto {
optional string host = 2;
optional int32 rpc_port = 3;
optional string tracking_url = 4;
- optional string diagnostics = 5 [default = "N/A"];
- optional YarnApplicationAttemptStateProto yarn_application_attempt_state = 6;
- optional ContainerIdProto am_container_id = 7;
+ optional string original_tracking_url = 5;
+ optional string diagnostics = 6 [default = "N/A"];
+ optional YarnApplicationAttemptStateProto yarn_application_attempt_state = 7;
+ optional ContainerIdProto am_container_id = 8;
}
enum NodeStateProto {
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml
index 56b9981..b21ea52 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml
@@ -57,14 +57,6 @@
log4j
log4j
-
- org.mortbay.jetty
- jetty-util
-
-
- com.sun.jersey
- jersey-client
-
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
deleted file mode 100644
index de1d3e2..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.client.api;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.service.AbstractService;
-import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
-import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
-import org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
-
-/**
- * A client library that can be used to post some information in terms of a
- * number of conceptual entities.
- */
-@Public
-@Unstable
-public abstract class TimelineClient extends AbstractService {
-
- @Public
- public static TimelineClient createTimelineClient() {
- TimelineClient client = new TimelineClientImpl();
- return client;
- }
-
- @Private
- protected TimelineClient(String name) {
- super(name);
- }
-
- /**
- *
- * Send the information of a number of conceptual entities to the timeline
- * server. It is a blocking API. The method will not return until it gets the
- * response from the timeline server.
- *
- *
- * @param entities
- * the collection of {@link TimelineEntity}
- * @return the error information if the sent entities are not correctly stored
- * @throws IOException
- * @throws YarnException
- */
- @Public
- public abstract TimelinePutResponse putEntities(
- TimelineEntity... entities) throws IOException, YarnException;
-
- /**
- *
- * Get a delegation token so as to be able to talk to the timeline server in a
- * secure way.
- *
- *
- * @param renewer
- * Address of the renewer who can renew these tokens when needed by
- * securely talking to the timeline server
- * @return a delegation token ({@link Token}) that can be used to talk to the
- * timeline server
- * @throws IOException
- * @throws YarnException
- */
- @Public
- public abstract Token getDelegationToken(
- String renewer) throws IOException, YarnException;
-
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineAuthenticator.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineAuthenticator.java
deleted file mode 100644
index 25333c7..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineAuthenticator.java
+++ /dev/null
@@ -1,260 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.client.api.impl;
-
-import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.net.HttpURLConnection;
-import java.net.URL;
-import java.net.URLEncoder;
-import java.text.MessageFormat;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
-import org.apache.hadoop.security.authentication.client.AuthenticationException;
-import org.apache.hadoop.security.authentication.client.Authenticator;
-import org.apache.hadoop.security.authentication.client.KerberosAuthenticator;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.yarn.api.records.timeline.TimelineDelegationTokenResponse;
-import org.apache.hadoop.yarn.security.client.TimelineAuthenticationConsts;
-import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
-import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenOperation;
-import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
-import org.codehaus.jackson.JsonNode;
-import org.codehaus.jackson.map.ObjectMapper;
-
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- * A KerberosAuthenticator subclass that fallback to
- * {@link TimelineAuthenticationConsts}.
- */
-@Private
-@Unstable
-public class TimelineAuthenticator extends KerberosAuthenticator {
-
- private static ObjectMapper mapper;
-
- static {
- mapper = new ObjectMapper();
- YarnJacksonJaxbJsonProvider.configObjectMapper(mapper);
- }
-
- /**
- * Returns the fallback authenticator if the server does not use Kerberos
- * SPNEGO HTTP authentication.
- *
- * @return a {@link TimelineAuthenticationConsts} instance.
- */
- @Override
- protected Authenticator getFallBackAuthenticator() {
- return new TimelineAuthenticator();
- }
-
- public static void injectDelegationToken(Map params,
- Token> dtToken)
- throws IOException {
- if (dtToken != null) {
- params.put(TimelineAuthenticationConsts.DELEGATION_PARAM,
- dtToken.encodeToUrlString());
- }
- }
-
- @Private
- @VisibleForTesting
- boolean hasDelegationToken(URL url) {
- if (url.getQuery() == null) {
- return false;
- } else {
- return url.getQuery().contains(
- TimelineAuthenticationConsts.DELEGATION_PARAM + "=");
- }
- }
-
- @Override
- public void authenticate(URL url, AuthenticatedURL.Token token)
- throws IOException, AuthenticationException {
- if (!hasDelegationToken(url)) {
- super.authenticate(url, token);
- }
- }
-
- public static Token getDelegationToken(
- URL url, AuthenticatedURL.Token token, String renewer) throws IOException {
- TimelineDelegationTokenOperation op =
- TimelineDelegationTokenOperation.GETDELEGATIONTOKEN;
- Map params = new HashMap();
- params.put(TimelineAuthenticationConsts.OP_PARAM, op.toString());
- params.put(TimelineAuthenticationConsts.RENEWER_PARAM, renewer);
- url = appendParams(url, params);
- AuthenticatedURL aUrl =
- new AuthenticatedURL(new TimelineAuthenticator());
- try {
- HttpURLConnection conn = aUrl.openConnection(url, token);
- conn.setRequestMethod(op.getHttpMethod());
- TimelineDelegationTokenResponse dtRes = validateAndParseResponse(conn);
- if (!dtRes.getType().equals(
- TimelineAuthenticationConsts.DELEGATION_TOKEN_URL)) {
- throw new IOException("The response content is not expected: "
- + dtRes.getContent());
- }
- String tokenStr = dtRes.getContent().toString();
- Token dToken =
- new Token();
- dToken.decodeFromUrlString(tokenStr);
- return dToken;
- } catch (AuthenticationException ex) {
- throw new IOException(ex.toString(), ex);
- }
- }
-
- public static long renewDelegationToken(URL url,
- AuthenticatedURL.Token token,
- Token dToken) throws IOException {
- Map params = new HashMap();
- params.put(TimelineAuthenticationConsts.OP_PARAM,
- TimelineDelegationTokenOperation.RENEWDELEGATIONTOKEN.toString());
- params.put(TimelineAuthenticationConsts.TOKEN_PARAM,
- dToken.encodeToUrlString());
- url = appendParams(url, params);
- AuthenticatedURL aUrl =
- new AuthenticatedURL(new TimelineAuthenticator());
- try {
- HttpURLConnection conn = aUrl.openConnection(url, token);
- conn.setRequestMethod(
- TimelineDelegationTokenOperation.RENEWDELEGATIONTOKEN.getHttpMethod());
- TimelineDelegationTokenResponse dtRes = validateAndParseResponse(conn);
- if (!dtRes.getType().equals(
- TimelineAuthenticationConsts.DELEGATION_TOKEN_EXPIRATION_TIME)) {
- throw new IOException("The response content is not expected: "
- + dtRes.getContent());
- }
- return Long.valueOf(dtRes.getContent().toString());
- } catch (AuthenticationException ex) {
- throw new IOException(ex.toString(), ex);
- }
- }
-
- public static void cancelDelegationToken(URL url,
- AuthenticatedURL.Token token,
- Token dToken) throws IOException {
- Map params = new HashMap();
- params.put(TimelineAuthenticationConsts.OP_PARAM,
- TimelineDelegationTokenOperation.CANCELDELEGATIONTOKEN.toString());
- params.put(TimelineAuthenticationConsts.TOKEN_PARAM,
- dToken.encodeToUrlString());
- url = appendParams(url, params);
- AuthenticatedURL aUrl =
- new AuthenticatedURL(new TimelineAuthenticator());
- try {
- HttpURLConnection conn = aUrl.openConnection(url, token);
- conn.setRequestMethod(TimelineDelegationTokenOperation.CANCELDELEGATIONTOKEN
- .getHttpMethod());
- validateAndParseResponse(conn);
- } catch (AuthenticationException ex) {
- throw new IOException(ex.toString(), ex);
- }
- }
-
- /**
- * Convenience method that appends parameters an HTTP URL.
- *
- * @param url
- * the url.
- * @param params
- * the query string parameters.
- *
- * @return a URL
- *
- * @throws IOException
- * thrown if an IO error occurs.
- */
- public static URL appendParams(URL url, Map params)
- throws IOException {
- StringBuilder sb = new StringBuilder();
- sb.append(url);
- String separator = url.toString().contains("?") ? "&" : "?";
- for (Map.Entry entry : params.entrySet()) {
- sb.append(separator).append(entry.getKey()).append("=").
- append(URLEncoder.encode(entry.getValue(), "UTF8"));
- separator = "&";
- }
- return new URL(sb.toString());
- }
-
- /**
- * Validates the response of an HttpURLConnection. If the current
- * status code is not 200, it will throw an exception with a detail message
- * using Server side error messages if available. Otherwise,
- * {@link TimelineDelegationTokenResponse} will be parsed and returned.
- *
- * @param conn
- * the HttpURLConnection.
- * @return
- * @throws IOException
- * thrown if the current status code is not 200 or the JSON response
- * cannot be parsed correctly
- */
- private static TimelineDelegationTokenResponse validateAndParseResponse(
- HttpURLConnection conn) throws IOException {
- int status = conn.getResponseCode();
- JsonNode json = mapper.readTree(conn.getInputStream());
- if (status == HttpURLConnection.HTTP_OK) {
- return mapper.readValue(json, TimelineDelegationTokenResponse.class);
- } else {
- // If the status code is not 200, some thing wrong should happen at the
- // server side, the JSON content is going to contain exception details.
- // We can use the JSON content to reconstruct the exception object.
- try {
- String message =
- json.get(TimelineAuthenticationConsts.ERROR_MESSAGE_JSON)
- .getTextValue();
- String exception =
- json.get(TimelineAuthenticationConsts.ERROR_EXCEPTION_JSON)
- .getTextValue();
- String className =
- json.get(TimelineAuthenticationConsts.ERROR_CLASSNAME_JSON)
- .getTextValue();
-
- try {
- ClassLoader cl = TimelineAuthenticator.class.getClassLoader();
- Class> klass = cl.loadClass(className);
- Constructor> constr = klass.getConstructor(String.class);
- throw (IOException) constr.newInstance(message);
- } catch (IOException ex) {
- throw ex;
- } catch (Exception ex) {
- throw new IOException(MessageFormat.format("{0} - {1}", exception,
- message));
- }
- } catch (IOException ex) {
- if (ex.getCause() instanceof IOException) {
- throw (IOException) ex.getCause();
- }
- throw new IOException(
- MessageFormat.format("HTTP status [{0}], {1}",
- status, conn.getResponseMessage()));
- }
- }
- }
-
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
deleted file mode 100644
index 5ffe17a..0000000
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
+++ /dev/null
@@ -1,314 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.client.api.impl;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.HttpURLConnection;
-import java.net.URI;
-import java.net.URL;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-
-import javax.ws.rs.core.MediaType;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.GnuParser;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Options;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
-import org.apache.hadoop.security.authentication.client.AuthenticationException;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
-import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
-import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
-import org.apache.hadoop.yarn.client.api.TimelineClient;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
-import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenSelector;
-import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
-import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
-import org.codehaus.jackson.map.ObjectMapper;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Joiner;
-import com.sun.jersey.api.client.Client;
-import com.sun.jersey.api.client.ClientResponse;
-import com.sun.jersey.api.client.WebResource;
-import com.sun.jersey.api.client.config.ClientConfig;
-import com.sun.jersey.api.client.config.DefaultClientConfig;
-import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
-import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
-
-@Private
-@Unstable
-public class TimelineClientImpl extends TimelineClient {
-
- private static final Log LOG = LogFactory.getLog(TimelineClientImpl.class);
- private static final String RESOURCE_URI_STR = "/ws/v1/timeline/";
- private static final Joiner JOINER = Joiner.on("");
- private static Options opts;
- static {
- opts = new Options();
- opts.addOption("put", true, "Put the TimelineEntities in a JSON file");
- opts.getOption("put").setArgName("Path to the JSON file");
- opts.addOption("help", false, "Print usage");
- }
-
- private Client client;
- private URI resURI;
- private boolean isEnabled;
- private TimelineAuthenticatedURLConnectionFactory urlFactory;
-
- public TimelineClientImpl() {
- super(TimelineClientImpl.class.getName());
- ClientConfig cc = new DefaultClientConfig();
- cc.getClasses().add(YarnJacksonJaxbJsonProvider.class);
- if (UserGroupInformation.isSecurityEnabled()) {
- urlFactory = new TimelineAuthenticatedURLConnectionFactory();
- client = new Client(new URLConnectionClientHandler(urlFactory), cc);
- } else {
- client = Client.create(cc);
- }
- }
-
- protected void serviceInit(Configuration conf) throws Exception {
- isEnabled = conf.getBoolean(
- YarnConfiguration.TIMELINE_SERVICE_ENABLED,
- YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED);
- if (!isEnabled) {
- LOG.info("Timeline service is not enabled");
- } else {
- if (YarnConfiguration.useHttps(conf)) {
- resURI = URI
- .create(JOINER.join("https://", conf.get(
- YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS,
- YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS),
- RESOURCE_URI_STR));
- } else {
- resURI = URI.create(JOINER.join("http://", conf.get(
- YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
- YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS),
- RESOURCE_URI_STR));
- }
- if (UserGroupInformation.isSecurityEnabled()) {
- urlFactory.setService(TimelineUtils.buildTimelineTokenService(conf));
- }
- LOG.info("Timeline service address: " + resURI);
- }
- super.serviceInit(conf);
- }
-
- @Override
- public TimelinePutResponse putEntities(
- TimelineEntity... entities) throws IOException, YarnException {
- if (!isEnabled) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Nothing will be put because timeline service is not enabled");
- }
- return new TimelinePutResponse();
- }
- TimelineEntities entitiesContainer = new TimelineEntities();
- entitiesContainer.addEntities(Arrays.asList(entities));
- ClientResponse resp;
- try {
- resp = doPostingEntities(entitiesContainer);
- } catch (RuntimeException re) {
- // runtime exception is expected if the client cannot connect the server
- String msg =
- "Failed to get the response from the timeline server.";
- LOG.error(msg, re);
- throw re;
- }
- if (resp == null ||
- resp.getClientResponseStatus() != ClientResponse.Status.OK) {
- String msg =
- "Failed to get the response from the timeline server.";
- LOG.error(msg);
- if (LOG.isDebugEnabled() && resp != null) {
- String output = resp.getEntity(String.class);
- LOG.debug("HTTP error code: " + resp.getStatus()
- + " Server response : \n" + output);
- }
- throw new YarnException(msg);
- }
- return resp.getEntity(TimelinePutResponse.class);
- }
-
- @Override
- public Token getDelegationToken(
- String renewer) throws IOException, YarnException {
- return TimelineAuthenticator.getDelegationToken(resURI.toURL(),
- urlFactory.token, renewer);
- }
-
- @Private
- @VisibleForTesting
- public ClientResponse doPostingEntities(TimelineEntities entities) {
- WebResource webResource = client.resource(resURI);
- return webResource.accept(MediaType.APPLICATION_JSON)
- .type(MediaType.APPLICATION_JSON)
- .post(ClientResponse.class, entities);
- }
-
- private static class TimelineAuthenticatedURLConnectionFactory
- implements HttpURLConnectionFactory {
-
- private AuthenticatedURL.Token token;
- private TimelineAuthenticator authenticator;
- private Token dToken;
- private Text service;
-
- public TimelineAuthenticatedURLConnectionFactory() {
- token = new AuthenticatedURL.Token();
- authenticator = new TimelineAuthenticator();
- }
-
- @Override
- public HttpURLConnection getHttpURLConnection(URL url) throws IOException {
- try {
- if (dToken == null) {
- //TODO: need to take care of the renew case
- dToken = selectToken();
- if (LOG.isDebugEnabled()) {
- LOG.debug("Timeline delegation token: " + dToken.toString());
- }
- }
- if (dToken != null) {
- Map params = new HashMap();
- TimelineAuthenticator.injectDelegationToken(params, dToken);
- url = TimelineAuthenticator.appendParams(url, params);
- if (LOG.isDebugEnabled()) {
- LOG.debug("URL with delegation token: " + url);
- }
- }
- return new AuthenticatedURL(authenticator).openConnection(url, token);
- } catch (AuthenticationException e) {
- LOG.error("Authentication failed when openning connection [" + url
- + "] with token [" + token + "].", e);
- throw new IOException(e);
- }
- }
-
- private Token selectToken() {
- UserGroupInformation ugi;
- try {
- ugi = UserGroupInformation.getCurrentUser();
- } catch (IOException e) {
- String msg = "Error when getting the current user";
- LOG.error(msg, e);
- throw new YarnRuntimeException(msg, e);
- }
- TimelineDelegationTokenSelector tokenSelector =
- new TimelineDelegationTokenSelector();
- return tokenSelector.selectToken(
- service, ugi.getCredentials().getAllTokens());
- }
-
- public void setService(Text service) {
- this.service = service;
- }
-
- }
-
- public static void main(String[] argv) throws Exception {
- CommandLine cliParser = new GnuParser().parse(opts, argv);
- if (cliParser.hasOption("put")) {
- String path = cliParser.getOptionValue("put");
- if (path != null && path.length() > 0) {
- putTimelineEntitiesInJSONFile(path);
- return;
- }
- }
- printUsage();
- }
-
- /**
- * Put timeline data in a JSON file via command line.
- *
- * @param path
- * path to the {@link TimelineEntities} JSON file
- */
- private static void putTimelineEntitiesInJSONFile(String path) {
- File jsonFile = new File(path);
- if (!jsonFile.exists()) {
- System.out.println("Error: File [" + jsonFile.getAbsolutePath()
- + "] doesn't exist");
- return;
- }
- ObjectMapper mapper = new ObjectMapper();
- YarnJacksonJaxbJsonProvider.configObjectMapper(mapper);
- TimelineEntities entities = null;
- try {
- entities = mapper.readValue(jsonFile, TimelineEntities.class);
- } catch (Exception e) {
- System.err.println("Error: " + e.getMessage());
- e.printStackTrace(System.err);
- return;
- }
- Configuration conf = new YarnConfiguration();
- TimelineClient client = TimelineClient.createTimelineClient();
- client.init(conf);
- client.start();
- try {
- if (UserGroupInformation.isSecurityEnabled()
- && conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, false)) {
- Token token =
- client.getDelegationToken(
- UserGroupInformation.getCurrentUser().getUserName());
- UserGroupInformation.getCurrentUser().addToken(token);
- }
- TimelinePutResponse response = client.putEntities(
- entities.getEntities().toArray(
- new TimelineEntity[entities.getEntities().size()]));
- if (response.getErrors().size() == 0) {
- System.out.println("Timeline data is successfully put");
- } else {
- for (TimelinePutResponse.TimelinePutError error : response.getErrors()) {
- System.out.println("TimelineEntity [" + error.getEntityType() + ":" +
- error.getEntityId() + "] is not successfully put. Error code: " +
- error.getErrorCode());
- }
- }
- } catch (Exception e) {
- System.err.println("Error: " + e.getMessage());
- e.printStackTrace(System.err);
- } finally {
- client.stop();
- }
- }
-
- /**
- * Helper function to print out usage
- */
- private static void printUsage() {
- new HelpFormatter().printHelp("TimelineClient", opts);
- }
-
-}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java
index 15bfa28..d82fa4a 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java
@@ -675,7 +675,7 @@ public QueueInfo createFakeQueueInfo() {
public ApplicationAttemptReport createFakeApplicationAttemptReport() {
return ApplicationAttemptReport.newInstance(
- createFakeApplicationAttemptId(), "localhost", 0, "", "",
+ createFakeApplicationAttemptId(), "localhost", 0, "", "", "",
YarnApplicationAttemptState.RUNNING, createFakeContainerId());
}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSClient.java
index 797faa5..d3c182b 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSClient.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSClient.java
@@ -346,6 +346,7 @@ private void createAppReports() {
"host",
124,
"url",
+ "oUrl",
"diagnostics",
YarnApplicationAttemptState.FINISHED,
ContainerId.newInstance(
@@ -357,6 +358,7 @@ private void createAppReports() {
"host",
124,
"url",
+ "oUrl",
"diagnostics",
YarnApplicationAttemptState.FINISHED,
ContainerId.newInstance(
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
index 6407f7a..8259893 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
@@ -457,6 +457,7 @@ public void setYarnApplicationState(YarnApplicationState state) {
"host",
124,
"url",
+ "oUrl",
"diagnostics",
YarnApplicationAttemptState.FINISHED,
ContainerId.newInstance(
@@ -467,6 +468,7 @@ public void setYarnApplicationState(YarnApplicationState state) {
"host",
124,
"url",
+ "oUrl",
"diagnostics",
YarnApplicationAttemptState.FINISHED,
ContainerId.newInstance(
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
index b408b61..66a54dd 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
@@ -127,7 +127,7 @@ public void testGetApplicationAttemptReport() throws Exception {
ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(
applicationId, 1);
ApplicationAttemptReport attemptReport = ApplicationAttemptReport
- .newInstance(attemptId, "host", 124, "url", "diagnostics",
+ .newInstance(attemptId, "host", 124, "url", "oUrl", "diagnostics",
YarnApplicationAttemptState.FINISHED, ContainerId.newInstance(
attemptId, 1));
when(
@@ -163,11 +163,11 @@ public void testGetApplicationAttempts() throws Exception {
ApplicationAttemptId attemptId1 = ApplicationAttemptId.newInstance(
applicationId, 2);
ApplicationAttemptReport attemptReport = ApplicationAttemptReport
- .newInstance(attemptId, "host", 124, "url", "diagnostics",
+ .newInstance(attemptId, "host", 124, "url", "oUrl", "diagnostics",
YarnApplicationAttemptState.FINISHED, ContainerId.newInstance(
attemptId, 1));
ApplicationAttemptReport attemptReport1 = ApplicationAttemptReport
- .newInstance(attemptId1, "host", 124, "url", "diagnostics",
+ .newInstance(attemptId1, "host", 124, "url", "oUrl", "diagnostics",
YarnApplicationAttemptState.FINISHED, ContainerId.newInstance(
attemptId1, 1));
List reports = new ArrayList();
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
index aa12b3f..ddae3da 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
@@ -71,6 +71,14 @@
jersey-core
+ org.mortbay.jetty
+ jetty-util
+
+
+ com.sun.jersey
+ jersey-client
+
+
org.codehaus.jackson
jackson-core-asl
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationAttemptReportPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationAttemptReportPBImpl.java
index 8999987..c3c0221 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationAttemptReportPBImpl.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationAttemptReportPBImpl.java
@@ -88,6 +88,15 @@ public String getTrackingUrl() {
}
@Override
+ public String getOriginalTrackingUrl() {
+ ApplicationAttemptReportProtoOrBuilder p = viaProto ? proto : builder;
+ if (!p.hasOriginalTrackingUrl()) {
+ return null;
+ }
+ return p.getOriginalTrackingUrl();
+ }
+
+ @Override
public String getDiagnostics() {
ApplicationAttemptReportProtoOrBuilder p = viaProto ? proto : builder;
if (!p.hasDiagnostics()) {
@@ -161,6 +170,16 @@ public void setTrackingUrl(String url) {
}
@Override
+ public void setOriginalTrackingUrl(String oUrl) {
+ maybeInitBuilder();
+ if (oUrl == null) {
+ builder.clearOriginalTrackingUrl();
+ return;
+ }
+ builder.setOriginalTrackingUrl(oUrl);
+ }
+
+ @Override
public void setDiagnostics(String diagnostics) {
maybeInitBuilder();
if (diagnostics == null) {
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
new file mode 100644
index 0000000..de1d3e2
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.client.api;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
+import org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
+
+/**
+ * A client library that can be used to post some information in terms of a
+ * number of conceptual entities.
+ */
+@Public
+@Unstable
+public abstract class TimelineClient extends AbstractService {
+
+ @Public
+ public static TimelineClient createTimelineClient() {
+ TimelineClient client = new TimelineClientImpl();
+ return client;
+ }
+
+ @Private
+ protected TimelineClient(String name) {
+ super(name);
+ }
+
+ /**
+ *
+ * Send the information of a number of conceptual entities to the timeline
+ * server. It is a blocking API. The method will not return until it gets the
+ * response from the timeline server.
+ *
+ *
+ * @param entities
+ * the collection of {@link TimelineEntity}
+ * @return the error information if the sent entities are not correctly stored
+ * @throws IOException
+ * @throws YarnException
+ */
+ @Public
+ public abstract TimelinePutResponse putEntities(
+ TimelineEntity... entities) throws IOException, YarnException;
+
+ /**
+ *
+ * Get a delegation token so as to be able to talk to the timeline server in a
+ * secure way.
+ *
+ *
+ * @param renewer
+ * Address of the renewer who can renew these tokens when needed by
+ * securely talking to the timeline server
+ * @return a delegation token ({@link Token}) that can be used to talk to the
+ * timeline server
+ * @throws IOException
+ * @throws YarnException
+ */
+ @Public
+ public abstract Token getDelegationToken(
+ String renewer) throws IOException, YarnException;
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineAuthenticator.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineAuthenticator.java
new file mode 100644
index 0000000..25333c7
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineAuthenticator.java
@@ -0,0 +1,260 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.client.api.impl;
+
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.text.MessageFormat;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.security.authentication.client.Authenticator;
+import org.apache.hadoop.security.authentication.client.KerberosAuthenticator;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDelegationTokenResponse;
+import org.apache.hadoop.yarn.security.client.TimelineAuthenticationConsts;
+import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
+import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenOperation;
+import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
+import org.codehaus.jackson.JsonNode;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * A KerberosAuthenticator subclass that fallback to
+ * {@link TimelineAuthenticationConsts}.
+ */
+@Private
+@Unstable
+public class TimelineAuthenticator extends KerberosAuthenticator {
+
+ private static ObjectMapper mapper;
+
+ static {
+ mapper = new ObjectMapper();
+ YarnJacksonJaxbJsonProvider.configObjectMapper(mapper);
+ }
+
+ /**
+ * Returns the fallback authenticator if the server does not use Kerberos
+ * SPNEGO HTTP authentication.
+ *
+ * @return a {@link TimelineAuthenticationConsts} instance.
+ */
+ @Override
+ protected Authenticator getFallBackAuthenticator() {
+ return new TimelineAuthenticator();
+ }
+
+ public static void injectDelegationToken(Map params,
+ Token> dtToken)
+ throws IOException {
+ if (dtToken != null) {
+ params.put(TimelineAuthenticationConsts.DELEGATION_PARAM,
+ dtToken.encodeToUrlString());
+ }
+ }
+
+ @Private
+ @VisibleForTesting
+ boolean hasDelegationToken(URL url) {
+ if (url.getQuery() == null) {
+ return false;
+ } else {
+ return url.getQuery().contains(
+ TimelineAuthenticationConsts.DELEGATION_PARAM + "=");
+ }
+ }
+
+ @Override
+ public void authenticate(URL url, AuthenticatedURL.Token token)
+ throws IOException, AuthenticationException {
+ if (!hasDelegationToken(url)) {
+ super.authenticate(url, token);
+ }
+ }
+
+ public static Token getDelegationToken(
+ URL url, AuthenticatedURL.Token token, String renewer) throws IOException {
+ TimelineDelegationTokenOperation op =
+ TimelineDelegationTokenOperation.GETDELEGATIONTOKEN;
+ Map params = new HashMap();
+ params.put(TimelineAuthenticationConsts.OP_PARAM, op.toString());
+ params.put(TimelineAuthenticationConsts.RENEWER_PARAM, renewer);
+ url = appendParams(url, params);
+ AuthenticatedURL aUrl =
+ new AuthenticatedURL(new TimelineAuthenticator());
+ try {
+ HttpURLConnection conn = aUrl.openConnection(url, token);
+ conn.setRequestMethod(op.getHttpMethod());
+ TimelineDelegationTokenResponse dtRes = validateAndParseResponse(conn);
+ if (!dtRes.getType().equals(
+ TimelineAuthenticationConsts.DELEGATION_TOKEN_URL)) {
+ throw new IOException("The response content is not expected: "
+ + dtRes.getContent());
+ }
+ String tokenStr = dtRes.getContent().toString();
+ Token dToken =
+ new Token();
+ dToken.decodeFromUrlString(tokenStr);
+ return dToken;
+ } catch (AuthenticationException ex) {
+ throw new IOException(ex.toString(), ex);
+ }
+ }
+
+ public static long renewDelegationToken(URL url,
+ AuthenticatedURL.Token token,
+ Token dToken) throws IOException {
+ Map params = new HashMap();
+ params.put(TimelineAuthenticationConsts.OP_PARAM,
+ TimelineDelegationTokenOperation.RENEWDELEGATIONTOKEN.toString());
+ params.put(TimelineAuthenticationConsts.TOKEN_PARAM,
+ dToken.encodeToUrlString());
+ url = appendParams(url, params);
+ AuthenticatedURL aUrl =
+ new AuthenticatedURL(new TimelineAuthenticator());
+ try {
+ HttpURLConnection conn = aUrl.openConnection(url, token);
+ conn.setRequestMethod(
+ TimelineDelegationTokenOperation.RENEWDELEGATIONTOKEN.getHttpMethod());
+ TimelineDelegationTokenResponse dtRes = validateAndParseResponse(conn);
+ if (!dtRes.getType().equals(
+ TimelineAuthenticationConsts.DELEGATION_TOKEN_EXPIRATION_TIME)) {
+ throw new IOException("The response content is not expected: "
+ + dtRes.getContent());
+ }
+ return Long.valueOf(dtRes.getContent().toString());
+ } catch (AuthenticationException ex) {
+ throw new IOException(ex.toString(), ex);
+ }
+ }
+
+ public static void cancelDelegationToken(URL url,
+ AuthenticatedURL.Token token,
+ Token dToken) throws IOException {
+ Map params = new HashMap();
+ params.put(TimelineAuthenticationConsts.OP_PARAM,
+ TimelineDelegationTokenOperation.CANCELDELEGATIONTOKEN.toString());
+ params.put(TimelineAuthenticationConsts.TOKEN_PARAM,
+ dToken.encodeToUrlString());
+ url = appendParams(url, params);
+ AuthenticatedURL aUrl =
+ new AuthenticatedURL(new TimelineAuthenticator());
+ try {
+ HttpURLConnection conn = aUrl.openConnection(url, token);
+ conn.setRequestMethod(TimelineDelegationTokenOperation.CANCELDELEGATIONTOKEN
+ .getHttpMethod());
+ validateAndParseResponse(conn);
+ } catch (AuthenticationException ex) {
+ throw new IOException(ex.toString(), ex);
+ }
+ }
+
+ /**
+ * Convenience method that appends parameters an HTTP URL.
+ *
+ * @param url
+ * the url.
+ * @param params
+ * the query string parameters.
+ *
+ * @return a URL
+ *
+ * @throws IOException
+ * thrown if an IO error occurs.
+ */
+ public static URL appendParams(URL url, Map params)
+ throws IOException {
+ StringBuilder sb = new StringBuilder();
+ sb.append(url);
+ String separator = url.toString().contains("?") ? "&" : "?";
+ for (Map.Entry entry : params.entrySet()) {
+ sb.append(separator).append(entry.getKey()).append("=").
+ append(URLEncoder.encode(entry.getValue(), "UTF8"));
+ separator = "&";
+ }
+ return new URL(sb.toString());
+ }
+
+ /**
+ * Validates the response of an HttpURLConnection. If the current
+ * status code is not 200, it will throw an exception with a detail message
+ * using Server side error messages if available. Otherwise,
+ * {@link TimelineDelegationTokenResponse} will be parsed and returned.
+ *
+ * @param conn
+ * the HttpURLConnection.
+ * @return
+ * @throws IOException
+ * thrown if the current status code is not 200 or the JSON response
+ * cannot be parsed correctly
+ */
+ private static TimelineDelegationTokenResponse validateAndParseResponse(
+ HttpURLConnection conn) throws IOException {
+ int status = conn.getResponseCode();
+ JsonNode json = mapper.readTree(conn.getInputStream());
+ if (status == HttpURLConnection.HTTP_OK) {
+ return mapper.readValue(json, TimelineDelegationTokenResponse.class);
+ } else {
+ // If the status code is not 200, some thing wrong should happen at the
+ // server side, the JSON content is going to contain exception details.
+ // We can use the JSON content to reconstruct the exception object.
+ try {
+ String message =
+ json.get(TimelineAuthenticationConsts.ERROR_MESSAGE_JSON)
+ .getTextValue();
+ String exception =
+ json.get(TimelineAuthenticationConsts.ERROR_EXCEPTION_JSON)
+ .getTextValue();
+ String className =
+ json.get(TimelineAuthenticationConsts.ERROR_CLASSNAME_JSON)
+ .getTextValue();
+
+ try {
+ ClassLoader cl = TimelineAuthenticator.class.getClassLoader();
+ Class> klass = cl.loadClass(className);
+ Constructor> constr = klass.getConstructor(String.class);
+ throw (IOException) constr.newInstance(message);
+ } catch (IOException ex) {
+ throw ex;
+ } catch (Exception ex) {
+ throw new IOException(MessageFormat.format("{0} - {1}", exception,
+ message));
+ }
+ } catch (IOException ex) {
+ if (ex.getCause() instanceof IOException) {
+ throw (IOException) ex.getCause();
+ }
+ throw new IOException(
+ MessageFormat.format("HTTP status [{0}], {1}",
+ status, conn.getResponseMessage()));
+ }
+ }
+ }
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
new file mode 100644
index 0000000..5ffe17a
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
@@ -0,0 +1,314 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.client.api.impl;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URI;
+import java.net.URL;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.ws.rs.core.MediaType;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
+import org.apache.hadoop.yarn.client.api.TimelineClient;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
+import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenSelector;
+import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
+import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Joiner;
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.client.config.ClientConfig;
+import com.sun.jersey.api.client.config.DefaultClientConfig;
+import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
+import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
+
+@Private
+@Unstable
+public class TimelineClientImpl extends TimelineClient {
+
+ private static final Log LOG = LogFactory.getLog(TimelineClientImpl.class);
+ private static final String RESOURCE_URI_STR = "/ws/v1/timeline/";
+ private static final Joiner JOINER = Joiner.on("");
+ private static Options opts;
+ static {
+ opts = new Options();
+ opts.addOption("put", true, "Put the TimelineEntities in a JSON file");
+ opts.getOption("put").setArgName("Path to the JSON file");
+ opts.addOption("help", false, "Print usage");
+ }
+
+ private Client client;
+ private URI resURI;
+ private boolean isEnabled;
+ private TimelineAuthenticatedURLConnectionFactory urlFactory;
+
+ public TimelineClientImpl() {
+ super(TimelineClientImpl.class.getName());
+ ClientConfig cc = new DefaultClientConfig();
+ cc.getClasses().add(YarnJacksonJaxbJsonProvider.class);
+ if (UserGroupInformation.isSecurityEnabled()) {
+ urlFactory = new TimelineAuthenticatedURLConnectionFactory();
+ client = new Client(new URLConnectionClientHandler(urlFactory), cc);
+ } else {
+ client = Client.create(cc);
+ }
+ }
+
+ protected void serviceInit(Configuration conf) throws Exception {
+ isEnabled = conf.getBoolean(
+ YarnConfiguration.TIMELINE_SERVICE_ENABLED,
+ YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED);
+ if (!isEnabled) {
+ LOG.info("Timeline service is not enabled");
+ } else {
+ if (YarnConfiguration.useHttps(conf)) {
+ resURI = URI
+ .create(JOINER.join("https://", conf.get(
+ YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS,
+ YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS),
+ RESOURCE_URI_STR));
+ } else {
+ resURI = URI.create(JOINER.join("http://", conf.get(
+ YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
+ YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS),
+ RESOURCE_URI_STR));
+ }
+ if (UserGroupInformation.isSecurityEnabled()) {
+ urlFactory.setService(TimelineUtils.buildTimelineTokenService(conf));
+ }
+ LOG.info("Timeline service address: " + resURI);
+ }
+ super.serviceInit(conf);
+ }
+
+ @Override
+ public TimelinePutResponse putEntities(
+ TimelineEntity... entities) throws IOException, YarnException {
+ if (!isEnabled) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Nothing will be put because timeline service is not enabled");
+ }
+ return new TimelinePutResponse();
+ }
+ TimelineEntities entitiesContainer = new TimelineEntities();
+ entitiesContainer.addEntities(Arrays.asList(entities));
+ ClientResponse resp;
+ try {
+ resp = doPostingEntities(entitiesContainer);
+ } catch (RuntimeException re) {
+ // runtime exception is expected if the client cannot connect the server
+ String msg =
+ "Failed to get the response from the timeline server.";
+ LOG.error(msg, re);
+ throw re;
+ }
+ if (resp == null ||
+ resp.getClientResponseStatus() != ClientResponse.Status.OK) {
+ String msg =
+ "Failed to get the response from the timeline server.";
+ LOG.error(msg);
+ if (LOG.isDebugEnabled() && resp != null) {
+ String output = resp.getEntity(String.class);
+ LOG.debug("HTTP error code: " + resp.getStatus()
+ + " Server response : \n" + output);
+ }
+ throw new YarnException(msg);
+ }
+ return resp.getEntity(TimelinePutResponse.class);
+ }
+
+ @Override
+ public Token getDelegationToken(
+ String renewer) throws IOException, YarnException {
+ return TimelineAuthenticator.getDelegationToken(resURI.toURL(),
+ urlFactory.token, renewer);
+ }
+
+ @Private
+ @VisibleForTesting
+ public ClientResponse doPostingEntities(TimelineEntities entities) {
+ WebResource webResource = client.resource(resURI);
+ return webResource.accept(MediaType.APPLICATION_JSON)
+ .type(MediaType.APPLICATION_JSON)
+ .post(ClientResponse.class, entities);
+ }
+
+ private static class TimelineAuthenticatedURLConnectionFactory
+ implements HttpURLConnectionFactory {
+
+ private AuthenticatedURL.Token token;
+ private TimelineAuthenticator authenticator;
+ private Token dToken;
+ private Text service;
+
+ public TimelineAuthenticatedURLConnectionFactory() {
+ token = new AuthenticatedURL.Token();
+ authenticator = new TimelineAuthenticator();
+ }
+
+ @Override
+ public HttpURLConnection getHttpURLConnection(URL url) throws IOException {
+ try {
+ if (dToken == null) {
+ //TODO: need to take care of the renew case
+ dToken = selectToken();
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Timeline delegation token: " + dToken.toString());
+ }
+ }
+ if (dToken != null) {
+ Map params = new HashMap();
+ TimelineAuthenticator.injectDelegationToken(params, dToken);
+ url = TimelineAuthenticator.appendParams(url, params);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("URL with delegation token: " + url);
+ }
+ }
+ return new AuthenticatedURL(authenticator).openConnection(url, token);
+ } catch (AuthenticationException e) {
+ LOG.error("Authentication failed when openning connection [" + url
+ + "] with token [" + token + "].", e);
+ throw new IOException(e);
+ }
+ }
+
+ private Token selectToken() {
+ UserGroupInformation ugi;
+ try {
+ ugi = UserGroupInformation.getCurrentUser();
+ } catch (IOException e) {
+ String msg = "Error when getting the current user";
+ LOG.error(msg, e);
+ throw new YarnRuntimeException(msg, e);
+ }
+ TimelineDelegationTokenSelector tokenSelector =
+ new TimelineDelegationTokenSelector();
+ return tokenSelector.selectToken(
+ service, ugi.getCredentials().getAllTokens());
+ }
+
+ public void setService(Text service) {
+ this.service = service;
+ }
+
+ }
+
+ public static void main(String[] argv) throws Exception {
+ CommandLine cliParser = new GnuParser().parse(opts, argv);
+ if (cliParser.hasOption("put")) {
+ String path = cliParser.getOptionValue("put");
+ if (path != null && path.length() > 0) {
+ putTimelineEntitiesInJSONFile(path);
+ return;
+ }
+ }
+ printUsage();
+ }
+
+ /**
+ * Put timeline data in a JSON file via command line.
+ *
+ * @param path
+ * path to the {@link TimelineEntities} JSON file
+ */
+ private static void putTimelineEntitiesInJSONFile(String path) {
+ File jsonFile = new File(path);
+ if (!jsonFile.exists()) {
+ System.out.println("Error: File [" + jsonFile.getAbsolutePath()
+ + "] doesn't exist");
+ return;
+ }
+ ObjectMapper mapper = new ObjectMapper();
+ YarnJacksonJaxbJsonProvider.configObjectMapper(mapper);
+ TimelineEntities entities = null;
+ try {
+ entities = mapper.readValue(jsonFile, TimelineEntities.class);
+ } catch (Exception e) {
+ System.err.println("Error: " + e.getMessage());
+ e.printStackTrace(System.err);
+ return;
+ }
+ Configuration conf = new YarnConfiguration();
+ TimelineClient client = TimelineClient.createTimelineClient();
+ client.init(conf);
+ client.start();
+ try {
+ if (UserGroupInformation.isSecurityEnabled()
+ && conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, false)) {
+ Token token =
+ client.getDelegationToken(
+ UserGroupInformation.getCurrentUser().getUserName());
+ UserGroupInformation.getCurrentUser().addToken(token);
+ }
+ TimelinePutResponse response = client.putEntities(
+ entities.getEntities().toArray(
+ new TimelineEntity[entities.getEntities().size()]));
+ if (response.getErrors().size() == 0) {
+ System.out.println("Timeline data is successfully put");
+ } else {
+ for (TimelinePutResponse.TimelinePutError error : response.getErrors()) {
+ System.out.println("TimelineEntity [" + error.getEntityType() + ":" +
+ error.getEntityId() + "] is not successfully put. Error code: " +
+ error.getErrorCode());
+ }
+ }
+ } catch (Exception e) {
+ System.err.println("Error: " + e.getMessage());
+ e.printStackTrace(System.err);
+ } finally {
+ client.stop();
+ }
+ }
+
+ /**
+ * Helper function to print out usage
+ */
+ private static void printUsage() {
+ new HelpFormatter().printHelp("TimelineClient", opts);
+ }
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlBlock.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlBlock.java
index 8f885bb..6ee0d1c 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlBlock.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlBlock.java
@@ -21,6 +21,7 @@
import java.io.PrintWriter;
import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.webapp.MimeType;
import org.apache.hadoop.yarn.webapp.SubView;
import org.apache.hadoop.yarn.webapp.WebAppException;
@@ -81,4 +82,15 @@ public void renderPartial() {
* @param html the block to render
*/
protected abstract void render(Block html);
+
+ protected UserGroupInformation getCallerUGI() {
+ // Check for the authorization.
+ String remoteUser = request().getRemoteUser();
+ UserGroupInformation callerUGI = null;
+ if (remoteUser != null) {
+ callerUGI = UserGroupInformation.createRemoteUser(remoteUser);
+ }
+ return callerUGI;
+ }
+
}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 0c1628e..12626a7 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -1261,6 +1261,19 @@
org.apache.hadoop.yarn.server.applicationhistoryservice.FileSystemApplicationHistoryStore
+
+ The setting that controls whether yarn metrics is published on
+ the timeline server or not.
+ yarn.metrics-publisher.enabled
+ false
+
+
+
+ Number of worker threads that send the yarn metrics data.
+ yarn.metrics-publisher.multi-threaded-dispatcher.pool-size
+ 10
+
+
The interval that the yarn client library uses to poll the
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java
index b56a595..803dc01 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java
@@ -163,7 +163,7 @@ private ApplicationAttemptReport convertToApplicationAttemptReport(
ApplicationAttemptHistoryData appAttemptHistory) {
return ApplicationAttemptReport.newInstance(
appAttemptHistory.getApplicationAttemptId(), appAttemptHistory.getHost(),
- appAttemptHistory.getRPCPort(), appAttemptHistory.getTrackingURL(),
+ appAttemptHistory.getRPCPort(), appAttemptHistory.getTrackingURL(), null,
appAttemptHistory.getDiagnosticsInfo(),
appAttemptHistory.getYarnApplicationAttemptState(),
appAttemptHistory.getMasterContainerId());
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
new file mode 100644
index 0000000..ab7f5fb
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
@@ -0,0 +1,524 @@
+package org.apache.hadoop.yarn.server.applicationhistoryservice;
+
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.service.AbstractService;
+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;
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerReport;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants;
+import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
+import org.apache.hadoop.yarn.server.metrics.ContainerMetricsConstants;
+import org.apache.hadoop.yarn.server.timeline.NameValuePair;
+import org.apache.hadoop.yarn.server.timeline.TimelineDataManager;
+import org.apache.hadoop.yarn.server.timeline.TimelineReader.Field;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
+
+public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
+ implements
+ ApplicationHistoryManager {
+
+ private static final Log LOG = LogFactory
+ .getLog(ApplicationHistoryManagerOnTimelineStore.class);
+
+ private TimelineDataManager timelineDataManager;
+ private String serverHttpAddress;
+
+ public ApplicationHistoryManagerOnTimelineStore(
+ TimelineDataManager timelineDataManager) {
+ super(ApplicationHistoryManagerOnTimelineStore.class.getName());
+ this.timelineDataManager = timelineDataManager;
+ }
+
+ @Override
+ protected void serviceInit(Configuration conf) throws Exception {
+ serverHttpAddress = WebAppUtils.getHttpSchemePrefix(conf) +
+ WebAppUtils.getAHSWebAppURLWithoutScheme(conf);
+ super.serviceInit(conf);
+ }
+
+ @Override
+ public ApplicationReport getApplication(ApplicationId appId)
+ throws IOException {
+ TimelineEntity entity = null;
+ try {
+ entity =
+ timelineDataManager.getEntity(
+ ApplicationMetricsConstants.ENTITY_TYPE,
+ appId.toString(), EnumSet.allOf(Field.class),
+ UserGroupInformation.getCurrentUser());
+ } catch (YarnException e) {
+ throw new IOException(e);
+ }
+ if (entity == null) {
+ return null;
+ } else {
+ return generateApplicationReport(entity);
+ }
+ }
+
+ @Override
+ public Map getAllApplications()
+ throws IOException {
+ TimelineEntities entities = null;
+ try {
+ entities =
+ timelineDataManager.getEntities(
+ ApplicationMetricsConstants.ENTITY_TYPE, null, null, null, null,
+ null, null, Long.MAX_VALUE, EnumSet.allOf(Field.class),
+ UserGroupInformation.getCurrentUser());
+ } catch (YarnException e) {
+ throw new IOException(e);
+ }
+ Map apps =
+ new HashMap();
+ if (entities != null && entities.getEntities() != null) {
+ for (TimelineEntity entity : entities.getEntities()) {
+ ApplicationReport app = generateApplicationReport(entity);
+ apps.put(app.getApplicationId(), app);
+ }
+ }
+ return apps;
+ }
+
+ @Override
+ public Map
+ getApplicationAttempts(
+ ApplicationId appId) throws IOException {
+ TimelineEntities entities = null;
+ try {
+ entities =
+ timelineDataManager.getEntities(
+ AppAttemptMetricsConstants.ENTITY_TYPE,
+ new NameValuePair(
+ AppAttemptMetricsConstants.PARENT_PRIMARY_FILTER, appId
+ .toString()), null, null, null, null, null,
+ Long.MAX_VALUE, EnumSet.allOf(Field.class),
+ UserGroupInformation.getCurrentUser());
+ } catch (YarnException e) {
+ throw new IOException(e);
+ }
+ Map appAttempts =
+ new HashMap();
+ if (entities != null && entities.getEntities() != null) {
+ for (TimelineEntity entity : entities.getEntities()) {
+ ApplicationAttemptReport appAttempt =
+ convertToApplicationAttemptReport(entity);
+ appAttempts.put(appAttempt.getApplicationAttemptId(), appAttempt);
+ }
+ }
+ return appAttempts;
+ }
+
+ @Override
+ public ApplicationAttemptReport getApplicationAttempt(
+ ApplicationAttemptId appAttemptId) throws IOException {
+ TimelineEntity entity = null;
+ try {
+ entity =
+ timelineDataManager.getEntity(
+ AppAttemptMetricsConstants.ENTITY_TYPE,
+ appAttemptId.toString(), EnumSet.allOf(Field.class),
+ UserGroupInformation.getCurrentUser());
+ } catch (YarnException e) {
+ throw new IOException(e);
+ }
+ if (entity == null) {
+ return null;
+ } else {
+ return convertToApplicationAttemptReport(entity);
+ }
+ }
+
+ @Override
+ public ContainerReport getContainer(ContainerId containerId)
+ throws IOException {
+ ApplicationReport app =
+ getApplication(containerId.getApplicationAttemptId().getApplicationId());
+ if (app == null) {
+ return null;
+ }
+ TimelineEntity entity = null;
+ try {
+ entity =
+ timelineDataManager.getEntity(
+ ContainerMetricsConstants.ENTITY_TYPE,
+ containerId.toString(), EnumSet.allOf(Field.class),
+ UserGroupInformation.getCurrentUser());
+ } catch (YarnException e) {
+ throw new IOException(e);
+ }
+ if (entity == null) {
+ return null;
+ } else {
+ return convertToContainerReport(entity, serverHttpAddress, app.getUser());
+ }
+ }
+
+ @Override
+ public ContainerReport getAMContainer(ApplicationAttemptId appAttemptId)
+ throws IOException {
+ ApplicationAttemptReport appAttempt = getApplicationAttempt(appAttemptId);
+ if (appAttempt == null) {
+ return null;
+ } else {
+ return getContainer(appAttempt.getAMContainerId());
+ }
+ }
+
+ @Override
+ public Map getContainers(
+ ApplicationAttemptId appAttemptId) throws IOException {
+ ApplicationReport app =
+ getApplication(appAttemptId.getApplicationId());
+ if (app == null) {
+ return new HashMap();
+ }
+ TimelineEntities entities = null;
+ try {
+ entities =
+ timelineDataManager.getEntities(
+ ContainerMetricsConstants.ENTITY_TYPE,
+ new NameValuePair(
+ ContainerMetricsConstants.PARENT_PRIMARIY_FILTER,
+ appAttemptId.toString()), null, null, null,
+ null, null, Long.MAX_VALUE, EnumSet.allOf(Field.class),
+ UserGroupInformation.getCurrentUser());
+ } catch (YarnException e) {
+ throw new IOException(e);
+ }
+ Map containers =
+ new HashMap();
+ if (entities != null && entities.getEntities() != null) {
+ for (TimelineEntity entity : entities.getEntities()) {
+ ContainerReport container =
+ convertToContainerReport(entity, serverHttpAddress, app.getUser());
+ containers.put(container.getContainerId(), container);
+ }
+ }
+ return containers;
+ }
+
+ private static ApplicationReport convertToApplicationReport(
+ TimelineEntity entity) {
+ String user = null;
+ String queue = null;
+ String name = null;
+ String type = null;
+ long createdTime = 0;
+ long finishedTime = 0;
+ ApplicationAttemptId latestApplicationAttemptId = null;
+ String diagnosticsInfo = null;
+ FinalApplicationStatus finalStatus = FinalApplicationStatus.UNDEFINED;
+ YarnApplicationState state = null;
+ Map entityInfo = entity.getOtherInfo();
+ if (entityInfo != null) {
+ if (entityInfo.containsKey(ApplicationMetricsConstants.USER_ENTITY_INFO)) {
+ user =
+ entityInfo.get(ApplicationMetricsConstants.USER_ENTITY_INFO)
+ .toString();
+ }
+ if (entityInfo.containsKey(ApplicationMetricsConstants.QUEUE_ENTITY_INFO)) {
+ queue =
+ entityInfo.get(ApplicationMetricsConstants.QUEUE_ENTITY_INFO)
+ .toString();
+ }
+ if (entityInfo.containsKey(ApplicationMetricsConstants.NAME_ENTITY_INFO)) {
+ name =
+ entityInfo.get(ApplicationMetricsConstants.NAME_ENTITY_INFO)
+ .toString();
+ }
+ if (entityInfo.containsKey(ApplicationMetricsConstants.TYPE_ENTITY_INFO)) {
+ type =
+ entityInfo.get(ApplicationMetricsConstants.TYPE_ENTITY_INFO)
+ .toString();
+ }
+ }
+ List events = entity.getEvents();
+ if (events != null) {
+ for (TimelineEvent event : events) {
+ if (event.getEventType().equals(
+ ApplicationMetricsConstants.CREATED_EVENT_TYPE)) {
+ createdTime = event.getTimestamp();
+ } else if (event.getEventType().equals(
+ ApplicationMetricsConstants.FINISHED_EVENT_TYPE)) {
+ finishedTime = event.getTimestamp();
+ Map eventInfo = event.getEventInfo();
+ if (eventInfo == null) {
+ continue;
+ }
+ if (eventInfo
+ .containsKey(ApplicationMetricsConstants.LATEST_APP_ATTEMPT_EVENT_INFO)) {
+ latestApplicationAttemptId =
+ ConverterUtils
+ .toApplicationAttemptId(
+ eventInfo
+ .get(
+ ApplicationMetricsConstants.LATEST_APP_ATTEMPT_EVENT_INFO)
+ .toString());
+ }
+ if (eventInfo
+ .containsKey(ApplicationMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO)) {
+ diagnosticsInfo =
+ eventInfo.get(
+ ApplicationMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO)
+ .toString();
+ }
+ if (eventInfo
+ .containsKey(ApplicationMetricsConstants.FINAL_STATUS_EVENT_INFO)) {
+ finalStatus =
+ FinalApplicationStatus.valueOf(eventInfo.get(
+ ApplicationMetricsConstants.FINAL_STATUS_EVENT_INFO)
+ .toString());
+ }
+ if (eventInfo
+ .containsKey(ApplicationMetricsConstants.STATE_EVENT_INFO)) {
+ state =
+ YarnApplicationState.valueOf(eventInfo.get(
+ ApplicationMetricsConstants.STATE_EVENT_INFO).toString());
+ }
+ }
+ }
+ }
+ return ApplicationReport.newInstance(
+ ConverterUtils.toApplicationId(entity.getEntityId()),
+ latestApplicationAttemptId, user, queue, name,
+ null,
+ -1,
+ null,
+ state,
+ diagnosticsInfo,
+ null,
+ createdTime,
+ finishedTime,
+ finalStatus,
+ null,
+ null,
+ 100,
+ type,
+ null);
+ }
+
+ private static ApplicationAttemptReport convertToApplicationAttemptReport(
+ TimelineEntity entity) {
+ String host = null;
+ int rpcPort = -1;
+ ContainerId amContainerId = null;
+ String trackingUrl = null;
+ String originalTrackingUrl = null;
+ String diagnosticsInfo = null;
+ YarnApplicationAttemptState state = null;
+ List events = entity.getEvents();
+ if (events != null) {
+ for (TimelineEvent event : events) {
+ if (event.getEventType().equals(
+ AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE)) {
+ Map eventInfo = event.getEventInfo();
+ if (eventInfo == null) {
+ continue;
+ }
+ if (eventInfo.containsKey(AppAttemptMetricsConstants.HOST_EVENT_INFO)) {
+ host =
+ eventInfo.get(AppAttemptMetricsConstants.HOST_EVENT_INFO)
+ .toString();
+ }
+ if (eventInfo
+ .containsKey(AppAttemptMetricsConstants.RPC_PORT_EVENT_INFO)) {
+ rpcPort =
+ Integer.valueOf(eventInfo.get(
+ AppAttemptMetricsConstants.RPC_PORT_EVENT_INFO).toString());
+ }
+ if (eventInfo
+ .containsKey(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO)) {
+ amContainerId =
+ ConverterUtils.toContainerId(eventInfo.get(
+ AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO)
+ .toString());
+ }
+ } else if (event.getEventType().equals(
+ AppAttemptMetricsConstants.FINISHED_EVENT_TYPE)) {
+ Map eventInfo = event.getEventInfo();
+ if (eventInfo == null) {
+ continue;
+ }
+ if (eventInfo
+ .containsKey(AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO)) {
+ trackingUrl =
+ eventInfo.get(
+ AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO)
+ .toString();
+ }
+ if (eventInfo
+ .containsKey(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO)) {
+ originalTrackingUrl =
+ eventInfo
+ .get(
+ AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO)
+ .toString();
+ }
+ if (eventInfo
+ .containsKey(AppAttemptMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO)) {
+ diagnosticsInfo =
+ eventInfo.get(
+ AppAttemptMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO)
+ .toString();
+ }
+ if (eventInfo
+ .containsKey(AppAttemptMetricsConstants.STATE_EVENT_INFO)) {
+ state =
+ YarnApplicationAttemptState.valueOf(eventInfo.get(
+ AppAttemptMetricsConstants.STATE_EVENT_INFO)
+ .toString());
+ }
+ }
+ }
+ }
+ return ApplicationAttemptReport.newInstance(
+ ConverterUtils.toApplicationAttemptId(entity.getEntityId()),
+ host, rpcPort, trackingUrl, originalTrackingUrl, diagnosticsInfo,
+ state, amContainerId);
+ }
+
+ private static ContainerReport convertToContainerReport(
+ TimelineEntity entity, String serverHttpAddress, String user) {
+ int allocatedMem = 0;
+ int allocatedVcore = 0;
+ String allocatedHost = null;
+ int allocatedPort = -1;
+ int allocatedPriority = 0;
+ long createdTime = 0;
+ long finishedTime = 0;
+ String diagnosticsInfo = null;
+ int exitStatus = ContainerExitStatus.INVALID;
+ ContainerState state = null;
+ Map entityInfo = entity.getOtherInfo();
+ if (entityInfo != null) {
+ if (entityInfo
+ .containsKey(ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO)) {
+ allocatedMem =
+ Integer.valueOf(entityInfo.get(
+ ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO)
+ .toString());
+ }
+ if (entityInfo
+ .containsKey(ContainerMetricsConstants.ALLOCATED_VCORE_ENTITY_INFO)) {
+ allocatedVcore =
+ Integer.valueOf(entityInfo.get(
+ ContainerMetricsConstants.ALLOCATED_VCORE_ENTITY_INFO)
+ .toString());
+ }
+ if (entityInfo
+ .containsKey(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO)) {
+ allocatedHost =
+ entityInfo
+ .get(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO)
+ .toString();
+ }
+ if (entityInfo
+ .containsKey(ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO)) {
+ allocatedPort =
+ Integer.valueOf(entityInfo.get(
+ ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO)
+ .toString());
+ }
+ if (entityInfo
+ .containsKey(ContainerMetricsConstants.ALLOCATED_PRIORITY_ENTITY_INFO)) {
+ allocatedPriority =
+ Integer.valueOf(entityInfo.get(
+ ContainerMetricsConstants.ALLOCATED_PRIORITY_ENTITY_INFO)
+ .toString());
+ }
+ }
+ List events = entity.getEvents();
+ if (events != null) {
+ for (TimelineEvent event : events) {
+ if (event.getEventType().equals(
+ ContainerMetricsConstants.CREATED_EVENT_TYPE)) {
+ createdTime = event.getTimestamp();
+ } else if (event.getEventType().equals(
+ ContainerMetricsConstants.FINISHED_EVENT_TYPE)) {
+ finishedTime = event.getTimestamp();
+ Map eventInfo = event.getEventInfo();
+ if (eventInfo == null) {
+ continue;
+ }
+ if (eventInfo
+ .containsKey(ContainerMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO)) {
+ diagnosticsInfo =
+ eventInfo.get(
+ ContainerMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO)
+ .toString();
+ }
+ if (eventInfo
+ .containsKey(ContainerMetricsConstants.EXIT_STATUS_EVENT_INFO)) {
+ exitStatus =
+ Integer.valueOf(eventInfo.get(
+ ContainerMetricsConstants.EXIT_STATUS_EVENT_INFO)
+ .toString());
+ }
+ if (eventInfo
+ .containsKey(ContainerMetricsConstants.STATE_EVENT_INFO)) {
+ state =
+ ContainerState.valueOf(eventInfo.get(
+ ContainerMetricsConstants.STATE_EVENT_INFO).toString());
+ }
+ }
+ }
+ }
+ NodeId allocatedNode = NodeId.newInstance(allocatedHost, allocatedPort);
+ ContainerId containerId =
+ ConverterUtils.toContainerId(entity.getEntityId());
+ String logUrl = WebAppUtils.getAggregatedLogURL(
+ serverHttpAddress,
+ allocatedNode.toString(),
+ containerId.toString(),
+ containerId.toString(),
+ user);
+ return ContainerReport.newInstance(
+ ConverterUtils.toContainerId(entity.getEntityId()),
+ Resource.newInstance(allocatedMem, allocatedVcore),
+ NodeId.newInstance(allocatedHost, allocatedPort),
+ Priority.newInstance(allocatedPriority),
+ createdTime, finishedTime, diagnosticsInfo, logUrl, exitStatus, state);
+ }
+
+ private ApplicationReport generateApplicationReport(TimelineEntity entity)
+ throws IOException {
+ ApplicationReport app = convertToApplicationReport(entity);
+ if (app != null && app.getCurrentApplicationAttemptId() != null) {
+ ApplicationAttemptReport appAttempt =
+ getApplicationAttempt(app.getCurrentApplicationAttemptId());
+ if (appAttempt != null) {
+ app.setHost(appAttempt.getHost());
+ app.setRpcPort(appAttempt.getRpcPort());
+ app.setTrackingUrl(appAttempt.getTrackingUrl());
+ app.setOriginalTrackingUrl(appAttempt.getTrackingUrl());
+ }
+ }
+ return app;
+ }
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
index dfd8c29..9f4c12c 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
@@ -40,6 +40,7 @@
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.server.applicationhistoryservice.webapp.AHSWebApp;
import org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore;
+import org.apache.hadoop.yarn.server.timeline.TimelineDataManager;
import org.apache.hadoop.yarn.server.timeline.TimelineStore;
import org.apache.hadoop.yarn.server.timeline.security.TimelineACLsManager;
import org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilterInitializer;
@@ -65,6 +66,7 @@
protected TimelineStore timelineStore;
protected TimelineDelegationTokenSecretManagerService secretManagerService;
protected TimelineACLsManager timelineACLsManager;
+ protected TimelineDataManager timelineDataManager;
protected WebApp webApp;
public ApplicationHistoryServer() {
@@ -73,15 +75,16 @@ public ApplicationHistoryServer() {
@Override
protected void serviceInit(Configuration conf) throws Exception {
- historyManager = createApplicationHistory();
- ahsClientService = createApplicationHistoryClientService(historyManager);
- addService(ahsClientService);
- addService((Service) historyManager);
timelineStore = createTimelineStore(conf);
addIfService(timelineStore);
secretManagerService = createTimelineDelegationTokenSecretManagerService(conf);
addService(secretManagerService);
timelineACLsManager = createTimelineACLsManager(conf);
+ timelineDataManager = createTimelineDataManager(conf);
+ historyManager = createApplicationHistory();
+ ahsClientService = createApplicationHistoryClientService(historyManager);
+ addService(ahsClientService);
+ addService((Service) historyManager);
DefaultMetricsSystem.initialize("ApplicationHistoryServer");
JvmMetrics.initSingleton("ApplicationHistoryServer", null);
@@ -123,7 +126,7 @@ public ApplicationHistoryClientService getClientService() {
}
protected ApplicationHistoryManager createApplicationHistory() {
- return new ApplicationHistoryManagerImpl();
+ return new ApplicationHistoryManagerOnTimelineStore(timelineDataManager);
}
protected ApplicationHistoryManager getApplicationHistory() {
@@ -176,6 +179,10 @@ protected TimelineACLsManager createTimelineACLsManager(Configuration conf) {
return new TimelineACLsManager(conf);
}
+ protected TimelineDataManager createTimelineDataManager(Configuration conf) {
+ return new TimelineDataManager(timelineStore, timelineACLsManager);
+ }
+
protected void startWebApp() {
Configuration conf = getConfig();
// Play trick to make the customized filter will only be loaded by the
@@ -204,6 +211,7 @@ protected void startWebApp() {
ahsWebApp.setTimelineStore(timelineStore);
ahsWebApp.setTimelineDelegationTokenSecretManagerService(secretManagerService);
ahsWebApp.setTimelineACLsManager(timelineACLsManager);
+ ahsWebApp.setTimelineDataManager(timelineDataManager);
webApp =
WebApps
.$for("applicationhistory", ApplicationHistoryClientService.class,
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebApp.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebApp.java
index 9901eeb..d6d8e2d 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebApp.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebApp.java
@@ -22,6 +22,7 @@
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.yarn.server.api.ApplicationContext;
import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryManager;
+import org.apache.hadoop.yarn.server.timeline.TimelineDataManager;
import org.apache.hadoop.yarn.server.timeline.TimelineStore;
import org.apache.hadoop.yarn.server.timeline.security.TimelineACLsManager;
import org.apache.hadoop.yarn.server.timeline.security.TimelineDelegationTokenSecretManagerService;
@@ -39,6 +40,7 @@
private TimelineStore timelineStore;
private TimelineDelegationTokenSecretManagerService secretManagerService;
private TimelineACLsManager timelineACLsManager;
+ private TimelineDataManager timelineDataManager;
private static AHSWebApp instance = null;
@@ -94,6 +96,14 @@ public void setTimelineACLsManager(TimelineACLsManager timelineACLsManager) {
this.timelineACLsManager = timelineACLsManager;
}
+ public TimelineDataManager getTimelineDataManager() {
+ return timelineDataManager;
+ }
+
+ public void setTimelineDataManager(TimelineDataManager timelineDataManager) {
+ this.timelineDataManager = timelineDataManager;
+ }
+
@Override
public void setup() {
bind(YarnJacksonJaxbJsonProvider.class);
@@ -101,10 +111,9 @@ public void setup() {
bind(TimelineWebServices.class);
bind(GenericExceptionHandler.class);
bind(ApplicationContext.class).toInstance(applicationHistoryManager);
- bind(TimelineStore.class).toInstance(timelineStore);
bind(TimelineDelegationTokenSecretManagerService.class).toInstance(
secretManagerService);
- bind(TimelineACLsManager.class).toInstance(timelineACLsManager);
+ bind(TimelineDataManager.class).toInstance(timelineDataManager);
route("/", AHSController.class);
route(pajoin("/apps", APP_STATE), AHSController.class);
route(pajoin("/app", APPLICATION_ID), AHSController.class, "app");
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
new file mode 100644
index 0000000..2443a34
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
@@ -0,0 +1,289 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.timeline;
+
+import static org.apache.hadoop.yarn.util.StringHelper.CSV_JOINER;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.SortedSet;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.timeline.TimelineReader.Field;
+import org.apache.hadoop.yarn.server.timeline.security.TimelineACLsManager;
+import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
+
+public class TimelineDataManager {
+
+ private static final Log LOG = LogFactory.getLog(TimelineDataManager.class);
+
+ private TimelineStore store;
+ private TimelineACLsManager timelineACLsManager;
+
+ public TimelineDataManager(TimelineStore store,
+ TimelineACLsManager timelineACLsManager) {
+ this.store = store;
+ this.timelineACLsManager = timelineACLsManager;
+ }
+
+ public TimelineEntities getEntities(
+ String entityType,
+ NameValuePair primaryFilter,
+ Collection secondaryFilter,
+ Long windowStart,
+ Long windowEnd,
+ String fromId,
+ Long fromTs,
+ Long limit,
+ EnumSet fields,
+ UserGroupInformation callerUGI) throws YarnException, IOException {
+ TimelineEntities entities = null;
+ boolean modified = extendFields(fields);
+ entities = store.getEntities(
+ entityType,
+ limit,
+ windowStart,
+ windowEnd,
+ fromId,
+ fromTs,
+ primaryFilter,
+ secondaryFilter,
+ fields);
+ if (entities != null) {
+ Iterator entitiesItr =
+ entities.getEntities().iterator();
+ while (entitiesItr.hasNext()) {
+ TimelineEntity entity = entitiesItr.next();
+ try {
+ // check ACLs
+ if (!timelineACLsManager.checkAccess(callerUGI, entity)) {
+ entitiesItr.remove();
+ } else {
+ // clean up system data
+ if (modified) {
+ entity.setPrimaryFilters(null);
+ } else {
+ cleanupOwnerInfo(entity);
+ }
+ }
+ } catch (YarnException e) {
+ LOG.error("Error when verifying access for user " + callerUGI
+ + " on the events of the timeline entity "
+ + new EntityIdentifier(entity.getEntityId(),
+ entity.getEntityType()), e);
+ entitiesItr.remove();
+ }
+ }
+ }
+ if (entities == null) {
+ return new TimelineEntities();
+ }
+ return entities;
+ }
+
+ public TimelineEntity getEntity(
+ String entityType,
+ String entityId,
+ EnumSet fields,
+ UserGroupInformation callerUGI) throws YarnException, IOException {
+ TimelineEntity entity = null;
+ boolean modified = extendFields(fields);
+ entity =
+ store.getEntity(entityId, entityType, fields);
+ if (entity != null) {
+ // check ACLs
+ if (!timelineACLsManager.checkAccess(callerUGI, entity)) {
+ entity = null;
+ } else {
+ // clean up the system data
+ if (modified) {
+ entity.setPrimaryFilters(null);
+ } else {
+ cleanupOwnerInfo(entity);
+ }
+ }
+ }
+ return entity;
+ }
+
+ public TimelineEvents getEvents(
+ String entityType,
+ SortedSet entityIds,
+ SortedSet eventTypes,
+ Long windowStart,
+ Long windowEnd,
+ Long limit,
+ UserGroupInformation callerUGI) throws YarnException, IOException {
+ TimelineEvents events = null;
+ events = store.getEntityTimelines(
+ entityType,
+ entityIds,
+ limit,
+ windowStart,
+ windowEnd,
+ eventTypes);
+ if (events != null) {
+ Iterator eventsItr =
+ events.getAllEvents().iterator();
+ while (eventsItr.hasNext()) {
+ TimelineEvents.EventsOfOneEntity eventsOfOneEntity = eventsItr.next();
+ try {
+ TimelineEntity entity = store.getEntity(
+ eventsOfOneEntity.getEntityId(),
+ eventsOfOneEntity.getEntityType(),
+ EnumSet.of(Field.PRIMARY_FILTERS));
+ // check ACLs
+ if (!timelineACLsManager.checkAccess(callerUGI, entity)) {
+ eventsItr.remove();
+ }
+ } catch (Exception e) {
+ LOG.error("Error when verifying access for user " + callerUGI
+ + " on the events of the timeline entity "
+ + new EntityIdentifier(eventsOfOneEntity.getEntityId(),
+ eventsOfOneEntity.getEntityType()), e);
+ eventsItr.remove();
+ }
+ }
+ }
+ if (events == null) {
+ return new TimelineEvents();
+ }
+ return events;
+ }
+
+ public TimelinePutResponse postEntities(
+ TimelineEntities entities,
+ UserGroupInformation callerUGI) throws YarnException, IOException {
+ if (entities == null) {
+ return new TimelinePutResponse();
+ }
+ List entityIDs = new ArrayList();
+ TimelineEntities entitiesToPut = new TimelineEntities();
+ List errors =
+ new ArrayList();
+ for (TimelineEntity entity : entities.getEntities()) {
+ EntityIdentifier entityID =
+ new EntityIdentifier(entity.getEntityId(), entity.getEntityType());
+
+ // check if there is existing entity
+ TimelineEntity existingEntity = null;
+ try {
+ existingEntity =
+ store.getEntity(entityID.getId(), entityID.getType(),
+ EnumSet.of(Field.PRIMARY_FILTERS));
+ if (existingEntity != null
+ && !timelineACLsManager.checkAccess(callerUGI, existingEntity)) {
+ throw new YarnException("The timeline entity " + entityID
+ + " was not put by " + callerUGI + " before");
+ }
+ } catch (Exception e) {
+ // Skip the entity which already exists and was put by others
+ LOG.warn("Skip the timeline entity: " + entityID + ", because "
+ + e.getMessage());
+ TimelinePutResponse.TimelinePutError error =
+ new TimelinePutResponse.TimelinePutError();
+ error.setEntityId(entityID.getId());
+ error.setEntityType(entityID.getType());
+ error.setErrorCode(
+ TimelinePutResponse.TimelinePutError.ACCESS_DENIED);
+ errors.add(error);
+ continue;
+ }
+
+ // inject owner information for the access check if this is the first
+ // time to post the entity, in case it's the admin who is updating
+ // the timeline data.
+ try {
+ if (existingEntity == null) {
+ injectOwnerInfo(entity,
+ callerUGI == null ? "" : callerUGI.getShortUserName());
+ }
+ } catch (YarnException e) {
+ // Skip the entity which messes up the primary filter and record the
+ // error
+ LOG.warn("Skip the timeline entity: " + entityID + ", because "
+ + e.getMessage());
+ TimelinePutResponse.TimelinePutError error =
+ new TimelinePutResponse.TimelinePutError();
+ error.setEntityId(entityID.getId());
+ error.setEntityType(entityID.getType());
+ error.setErrorCode(
+ TimelinePutResponse.TimelinePutError.SYSTEM_FILTER_CONFLICT);
+ errors.add(error);
+ continue;
+ }
+
+ entityIDs.add(entityID);
+ entitiesToPut.addEntity(entity);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Storing the entity " + entityID + ", JSON-style content: "
+ + TimelineUtils.dumpTimelineRecordtoJSON(entity));
+ }
+ }
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Storing entities: " + CSV_JOINER.join(entityIDs));
+ }
+ TimelinePutResponse response = store.put(entitiesToPut);
+ // add the errors of timeline system filter key conflict
+ response.addErrors(errors);
+ return response;
+ }
+
+ private static boolean extendFields(EnumSet fieldEnums) {
+ boolean modified = false;
+ if (fieldEnums != null && !fieldEnums.contains(Field.PRIMARY_FILTERS)) {
+ fieldEnums.add(Field.PRIMARY_FILTERS);
+ modified = true;
+ }
+ return modified;
+ }
+
+ private static void injectOwnerInfo(TimelineEntity timelineEntity,
+ String owner) throws YarnException {
+ if (timelineEntity.getPrimaryFilters() != null &&
+ timelineEntity.getPrimaryFilters().containsKey(
+ TimelineStore.SystemFilter.ENTITY_OWNER.toString())) {
+ throw new YarnException(
+ "User should not use the timeline system filter key: "
+ + TimelineStore.SystemFilter.ENTITY_OWNER);
+ }
+ timelineEntity.addPrimaryFilter(
+ TimelineStore.SystemFilter.ENTITY_OWNER
+ .toString(), owner);
+ }
+
+ private static void cleanupOwnerInfo(TimelineEntity timelineEntity) {
+ if (timelineEntity.getPrimaryFilters() != null) {
+ timelineEntity.getPrimaryFilters().remove(
+ TimelineStore.SystemFilter.ENTITY_OWNER.toString());
+ }
+ }
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java
index a4e8d58..2e99ace 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java
@@ -18,14 +18,10 @@
package org.apache.hadoop.yarn.server.timeline.webapp;
-import static org.apache.hadoop.yarn.util.StringHelper.CSV_JOINER;
-
-import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.EnumSet;
import java.util.HashSet;
-import java.util.Iterator;
import java.util.List;
import java.util.Set;
import java.util.SortedSet;
@@ -58,14 +54,11 @@
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
-import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.server.timeline.EntityIdentifier;
import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
import org.apache.hadoop.yarn.server.timeline.NameValuePair;
-import org.apache.hadoop.yarn.server.timeline.TimelineStore;
+import org.apache.hadoop.yarn.server.timeline.TimelineDataManager;
import org.apache.hadoop.yarn.server.timeline.TimelineReader.Field;
-import org.apache.hadoop.yarn.server.timeline.security.TimelineACLsManager;
-import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
import org.apache.hadoop.yarn.webapp.BadRequestException;
import org.apache.hadoop.yarn.webapp.NotFoundException;
@@ -79,14 +72,11 @@
private static final Log LOG = LogFactory.getLog(TimelineWebServices.class);
- private TimelineStore store;
- private TimelineACLsManager timelineACLsManager;
+ private TimelineDataManager timelineDataManager;
@Inject
- public TimelineWebServices(TimelineStore store,
- TimelineACLsManager timelineACLsManager) {
- this.store = store;
- this.timelineACLsManager = timelineACLsManager;
+ public TimelineWebServices(TimelineDataManager timelineDataManager) {
+ this.timelineDataManager = timelineDataManager;
}
@XmlRootElement(name = "about")
@@ -147,61 +137,28 @@ public TimelineEntities getEntities(
@QueryParam("limit") String limit,
@QueryParam("fields") String fields) {
init(res);
- TimelineEntities entities = null;
try {
- EnumSet fieldEnums = parseFieldsStr(fields, ",");
- boolean modified = extendFields(fieldEnums);
- UserGroupInformation callerUGI = getUser(req);
- entities = store.getEntities(
+ return timelineDataManager.getEntities(
parseStr(entityType),
- parseLongStr(limit),
+ parsePairStr(primaryFilter, ":"),
+ parsePairsStr(secondaryFilter, ",", ":"),
parseLongStr(windowStart),
parseLongStr(windowEnd),
parseStr(fromId),
parseLongStr(fromTs),
- parsePairStr(primaryFilter, ":"),
- parsePairsStr(secondaryFilter, ",", ":"),
- fieldEnums);
- if (entities != null) {
- Iterator entitiesItr =
- entities.getEntities().iterator();
- while (entitiesItr.hasNext()) {
- TimelineEntity entity = entitiesItr.next();
- try {
- // check ACLs
- if (!timelineACLsManager.checkAccess(callerUGI, entity)) {
- entitiesItr.remove();
- } else {
- // clean up system data
- if (modified) {
- entity.setPrimaryFilters(null);
- } else {
- cleanupOwnerInfo(entity);
- }
- }
- } catch (YarnException e) {
- LOG.error("Error when verifying access for user " + callerUGI
- + " on the events of the timeline entity "
- + new EntityIdentifier(entity.getEntityId(),
- entity.getEntityType()), e);
- entitiesItr.remove();
- }
- }
- }
+ parseLongStr(limit),
+ parseFieldsStr(fields, ","),
+ getUser(req));
} catch (NumberFormatException e) {
throw new BadRequestException(
"windowStart, windowEnd or limit is not a numeric value.");
} catch (IllegalArgumentException e) {
throw new BadRequestException("requested invalid field.");
- } catch (IOException e) {
+ } catch (Exception e) {
LOG.error("Error getting entities", e);
throw new WebApplicationException(e,
Response.Status.INTERNAL_SERVER_ERROR);
}
- if (entities == null) {
- return new TimelineEntities();
- }
- return entities;
}
/**
@@ -219,33 +176,15 @@ public TimelineEntity getEntity(
init(res);
TimelineEntity entity = null;
try {
- EnumSet fieldEnums = parseFieldsStr(fields, ",");
- boolean modified = extendFields(fieldEnums);
- entity =
- store.getEntity(parseStr(entityId), parseStr(entityType),
- fieldEnums);
- if (entity != null) {
- // check ACLs
- UserGroupInformation callerUGI = getUser(req);
- if (!timelineACLsManager.checkAccess(callerUGI, entity)) {
- entity = null;
- } else {
- // clean up the system data
- if (modified) {
- entity.setPrimaryFilters(null);
- } else {
- cleanupOwnerInfo(entity);
- }
- }
- }
+ entity = timelineDataManager.getEntity(
+ parseStr(entityType),
+ parseStr(entityId),
+ parseFieldsStr(fields, ","),
+ getUser(req));
} catch (IllegalArgumentException e) {
throw new BadRequestException(
"requested invalid field.");
- } catch (IOException e) {
- LOG.error("Error getting entity", e);
- throw new WebApplicationException(e,
- Response.Status.INTERNAL_SERVER_ERROR);
- } catch (YarnException e) {
+ } catch (Exception e) {
LOG.error("Error getting entity", e);
throw new WebApplicationException(e,
Response.Status.INTERNAL_SERVER_ERROR);
@@ -274,51 +213,23 @@ public TimelineEvents getEvents(
@QueryParam("windowEnd") String windowEnd,
@QueryParam("limit") String limit) {
init(res);
- TimelineEvents events = null;
try {
- UserGroupInformation callerUGI = getUser(req);
- events = store.getEntityTimelines(
+ return timelineDataManager.getEvents(
parseStr(entityType),
parseArrayStr(entityId, ","),
- parseLongStr(limit),
+ parseArrayStr(eventType, ","),
parseLongStr(windowStart),
parseLongStr(windowEnd),
- parseArrayStr(eventType, ","));
- if (events != null) {
- Iterator eventsItr =
- events.getAllEvents().iterator();
- while (eventsItr.hasNext()) {
- TimelineEvents.EventsOfOneEntity eventsOfOneEntity = eventsItr.next();
- try {
- TimelineEntity entity = store.getEntity(
- eventsOfOneEntity.getEntityId(),
- eventsOfOneEntity.getEntityType(),
- EnumSet.of(Field.PRIMARY_FILTERS));
- // check ACLs
- if (!timelineACLsManager.checkAccess(callerUGI, entity)) {
- eventsItr.remove();
- }
- } catch (Exception e) {
- LOG.error("Error when verifying access for user " + callerUGI
- + " on the events of the timeline entity "
- + new EntityIdentifier(eventsOfOneEntity.getEntityId(),
- eventsOfOneEntity.getEntityType()), e);
- eventsItr.remove();
- }
- }
- }
+ parseLongStr(limit),
+ getUser(req));
} catch (NumberFormatException e) {
throw new BadRequestException(
"windowStart, windowEnd or limit is not a numeric value.");
- } catch (IOException e) {
+ } catch (Exception e) {
LOG.error("Error getting entity timelines", e);
throw new WebApplicationException(e,
Response.Status.INTERNAL_SERVER_ERROR);
}
- if (events == null) {
- return new TimelineEvents();
- }
- return events;
}
/**
@@ -332,82 +243,10 @@ public TimelinePutResponse postEntities(
@Context HttpServletResponse res,
TimelineEntities entities) {
init(res);
- if (entities == null) {
- return new TimelinePutResponse();
- }
UserGroupInformation callerUGI = getUser(req);
try {
- List entityIDs = new ArrayList();
- TimelineEntities entitiesToPut = new TimelineEntities();
- List errors =
- new ArrayList();
- for (TimelineEntity entity : entities.getEntities()) {
- EntityIdentifier entityID =
- new EntityIdentifier(entity.getEntityId(), entity.getEntityType());
-
- // check if there is existing entity
- TimelineEntity existingEntity = null;
- try {
- existingEntity =
- store.getEntity(entityID.getId(), entityID.getType(),
- EnumSet.of(Field.PRIMARY_FILTERS));
- if (existingEntity != null
- && !timelineACLsManager.checkAccess(callerUGI, existingEntity)) {
- throw new YarnException("The timeline entity " + entityID
- + " was not put by " + callerUGI + " before");
- }
- } catch (Exception e) {
- // Skip the entity which already exists and was put by others
- LOG.warn("Skip the timeline entity: " + entityID + ", because "
- + e.getMessage());
- TimelinePutResponse.TimelinePutError error =
- new TimelinePutResponse.TimelinePutError();
- error.setEntityId(entityID.getId());
- error.setEntityType(entityID.getType());
- error.setErrorCode(
- TimelinePutResponse.TimelinePutError.ACCESS_DENIED);
- errors.add(error);
- continue;
- }
-
- // inject owner information for the access check if this is the first
- // time to post the entity, in case it's the admin who is updating
- // the timeline data.
- try {
- if (existingEntity == null) {
- injectOwnerInfo(entity,
- callerUGI == null ? "" : callerUGI.getShortUserName());
- }
- } catch (YarnException e) {
- // Skip the entity which messes up the primary filter and record the
- // error
- LOG.warn("Skip the timeline entity: " + entityID + ", because "
- + e.getMessage());
- TimelinePutResponse.TimelinePutError error =
- new TimelinePutResponse.TimelinePutError();
- error.setEntityId(entityID.getId());
- error.setEntityType(entityID.getType());
- error.setErrorCode(
- TimelinePutResponse.TimelinePutError.SYSTEM_FILTER_CONFLICT);
- errors.add(error);
- continue;
- }
-
- entityIDs.add(entityID);
- entitiesToPut.addEntity(entity);
- if (LOG.isDebugEnabled()) {
- LOG.debug("Storing the entity " + entityID + ", JSON-style content: "
- + TimelineUtils.dumpTimelineRecordtoJSON(entity));
- }
- }
- if (LOG.isDebugEnabled()) {
- LOG.debug("Storing entities: " + CSV_JOINER.join(entityIDs));
- }
- TimelinePutResponse response = store.put(entitiesToPut);
- // add the errors of timeline system filter key conflict
- response.addErrors(errors);
- return response;
- } catch (IOException e) {
+ return timelineDataManager.postEntities(entities, callerUGI);
+ } catch (Exception e) {
LOG.error("Error putting entities", e);
throw new WebApplicationException(e,
Response.Status.INTERNAL_SERVER_ERROR);
@@ -418,6 +257,15 @@ private void init(HttpServletResponse response) {
response.setContentType(null);
}
+ private static UserGroupInformation getUser(HttpServletRequest req) {
+ String remoteUser = req.getRemoteUser();
+ UserGroupInformation callerUGI = null;
+ if (remoteUser != null) {
+ callerUGI = UserGroupInformation.createRemoteUser(remoteUser);
+ }
+ return callerUGI;
+ }
+
private static SortedSet parseArrayStr(String str, String delimiter) {
if (str == null) {
return null;
@@ -490,14 +338,6 @@ private static NameValuePair parsePairStr(String str, String delimiter) {
}
}
- private static boolean extendFields(EnumSet fieldEnums) {
- boolean modified = false;
- if (fieldEnums != null && !fieldEnums.contains(Field.PRIMARY_FILTERS)) {
- fieldEnums.add(Field.PRIMARY_FILTERS);
- modified = true;
- }
- return modified;
- }
private static Long parseLongStr(String str) {
return str == null ? null : Long.parseLong(str.trim());
}
@@ -506,34 +346,4 @@ private static String parseStr(String str) {
return str == null ? null : str.trim();
}
- private static UserGroupInformation getUser(HttpServletRequest req) {
- String remoteUser = req.getRemoteUser();
- UserGroupInformation callerUGI = null;
- if (remoteUser != null) {
- callerUGI = UserGroupInformation.createRemoteUser(remoteUser);
- }
- return callerUGI;
- }
-
- private static void injectOwnerInfo(TimelineEntity timelineEntity,
- String owner) throws YarnException {
- if (timelineEntity.getPrimaryFilters() != null &&
- timelineEntity.getPrimaryFilters().containsKey(
- TimelineStore.SystemFilter.ENTITY_OWNER.toString())) {
- throw new YarnException(
- "User should not use the timeline system filter key: "
- + TimelineStore.SystemFilter.ENTITY_OWNER);
- }
- timelineEntity.addPrimaryFilter(
- TimelineStore.SystemFilter.ENTITY_OWNER
- .toString(), owner);
- }
-
- private static void cleanupOwnerInfo(TimelineEntity timelineEntity) {
- if (timelineEntity.getPrimaryFilters() != null) {
- timelineEntity.getPrimaryFilters().remove(
- TimelineStore.SystemFilter.ENTITY_OWNER.toString());
- }
- }
-
}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/AppAttemptMetricsConstants.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/AppAttemptMetricsConstants.java
new file mode 100644
index 0000000..a7809cf
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/AppAttemptMetricsConstants.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.metrics;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+@Private
+@Unstable
+public class AppAttemptMetricsConstants {
+
+ public static final String ENTITY_TYPE =
+ "YARN_APPLICATION_ATTEMPT";
+
+ public static final String REGISTERED_EVENT_TYPE =
+ "YARN_APPLICATION_ATTEMPT_REGISTERED";
+
+ public static final String FINISHED_EVENT_TYPE =
+ "YARN_APPLICATION_ATTEMPT_FINISHED";
+
+ public static final String PARENT_PRIMARY_FILTER =
+ "YARN_APPLICATION_ATTEMPT_PARENT";
+
+ public static final String TRACKING_URL_EVENT_INFO =
+ "YARN_APPLICATION_ATTEMPT_TRACKING_URL";
+
+ public static final String ORIGINAL_TRACKING_URL_EVENT_INFO =
+ "YARN_APPLICATION_ATTEMPT_ORIGINAL_TRACKING_URL";
+
+ public static final String HOST_EVENT_INFO =
+ "YARN_APPLICATION_ATTEMPT_HOST";
+
+ public static final String RPC_PORT_EVENT_INFO =
+ "YARN_APPLICATION_ATTEMPT_RPC_PORT";
+
+ public static final String MASTER_CONTAINER_EVENT_INFO =
+ "YARN_APPLICATION_ATTEMPT_MASTER_CONTAINER";
+
+ public static final String DIAGNOSTICS_INFO_EVENT_INFO =
+ "YARN_APPLICATION_ATTEMPT_DIAGNOSTICS_INFO";
+
+ public static final String FINAL_STATUS_EVENT_INFO =
+ "YARN_APPLICATION_ATTEMPT_FINAL_STATUS";
+
+ public static final String STATE_EVENT_INFO =
+ "YARN_APPLICATION_ATTEMPT_STATE";
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ApplicationMetricsConstants.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ApplicationMetricsConstants.java
new file mode 100644
index 0000000..f6a40bd
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ApplicationMetricsConstants.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.metrics;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+@Private
+@Unstable
+public class ApplicationMetricsConstants {
+
+ public static final String ENTITY_TYPE =
+ "YARN_APPLICATION";
+
+ public static final String CREATED_EVENT_TYPE =
+ "YARN_APPLICATION_CREATED";
+
+ public static final String FINISHED_EVENT_TYPE =
+ "YARN_APPLICATION_FINISHED";
+
+ public static final String NAME_ENTITY_INFO =
+ "YARN_APPLICATION_NAME";
+
+ public static final String TYPE_ENTITY_INFO =
+ "YARN_APPLICATION_TYPE";
+
+ public static final String USER_ENTITY_INFO =
+ "YARN_APPLICATION_USER";
+
+ public static final String QUEUE_ENTITY_INFO =
+ "YARN_APPLICATION_QUEUE";
+
+ public static final String SUBMITTED_TIME_ENTITY_INFO =
+ "YARN_APPLICATION_SUBMITTED_TIME";
+
+ public static final String DIAGNOSTICS_INFO_EVENT_INFO =
+ "YARN_APPLICATION_DIAGNOSTICS_INFO";
+
+ public static final String FINAL_STATUS_EVENT_INFO =
+ "YARN_APPLICATION_FINAL_STATUS";
+
+ public static final String STATE_EVENT_INFO =
+ "YARN_APPLICATION_STATE";
+
+ public static final String LATEST_APP_ATTEMPT_EVENT_INFO =
+ "YARN_APPLICATION_LATEST_APP_ATTEMPT";
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ContainerMetricsConstants.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ContainerMetricsConstants.java
new file mode 100644
index 0000000..8791da4
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/ContainerMetricsConstants.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.metrics;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+@Private
+@Unstable
+public class ContainerMetricsConstants {
+
+ public static final String ENTITY_TYPE = "YARN_CONTAINER";
+
+ public static final String CREATED_EVENT_TYPE = "YARN_CONTAINER_CREATED";
+
+ public static final String FINISHED_EVENT_TYPE = "YARN_CONTAINER_FINISHED";
+
+ public static final String PARENT_PRIMARIY_FILTER = "YARN_CONTAINER_PARENT";
+
+ public static final String ALLOCATED_MEMORY_ENTITY_INFO =
+ "YARN_CONTAINER_ALLOCATED_MEMORY";
+
+ public static final String ALLOCATED_VCORE_ENTITY_INFO =
+ "YARN_CONTAINER_ALLOCATED_VCORE";
+
+ public static final String ALLOCATED_HOST_ENTITY_INFO =
+ "YARN_CONTAINER_ALLOCATED_HOST";
+
+ public static final String ALLOCATED_PORT_ENTITY_INFO =
+ "YARN_CONTAINER_ALLOCATED_PORT";
+
+ public static final String ALLOCATED_PRIORITY_ENTITY_INFO =
+ "YARN_CONTAINER_ALLOCATED_PRIORITY";
+
+ public static final String DIAGNOSTICS_INFO_EVENT_INFO =
+ "YARN_CONTAINER_DIAGNOSTICS_INFO";
+
+ public static final String EXIT_STATUS_EVENT_INFO =
+ "YARN_CONTAINER_EXIT_STATUS";
+
+ public static final String STATE_EVENT_INFO =
+ "YARN_CONTAINER_STATE";
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
index 1a76eca..3e37881 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
@@ -20,12 +20,13 @@
import static org.apache.hadoop.yarn.util.StringHelper.join;
import static org.apache.hadoop.yarn.webapp.YarnWebParams.APPLICATION_ATTEMPT_ID;
-import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
import java.util.Collection;
import org.apache.commons.lang.StringEscapeUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
import org.apache.hadoop.yarn.api.records.ContainerReport;
@@ -67,10 +68,22 @@ protected void render(Block html) {
return;
}
+ final ApplicationAttemptId appAttemptIdFinal = appAttemptId;
+ UserGroupInformation callerUGI = getCallerUGI();
ApplicationAttemptReport appAttemptReport;
try {
- appAttemptReport = appContext.getApplicationAttempt(appAttemptId);
- } catch (IOException e) {
+ if (callerUGI == null) {
+ appAttemptReport = appContext.getApplicationAttempt(appAttemptId);
+ } else {
+ appAttemptReport = callerUGI.doAs(
+ new PrivilegedExceptionAction () {
+ @Override
+ public ApplicationAttemptReport run() throws Exception {
+ return appContext.getApplicationAttempt(appAttemptIdFinal);
+ }
+ });
+ }
+ } catch (Exception e) {
String message =
"Failed to read the application attempt " + appAttemptId + ".";
LOG.error(message, e);
@@ -108,8 +121,18 @@ protected void render(Block html) {
Collection containers;
try {
- containers = appContext.getContainers(appAttemptId).values();
- } catch (IOException e) {
+ if (callerUGI == null) {
+ containers = appContext.getContainers(appAttemptId).values();
+ } else {
+ containers = callerUGI.doAs(
+ new PrivilegedExceptionAction> () {
+ @Override
+ public Collection run() throws Exception {
+ return appContext.getContainers(appAttemptIdFinal).values();
+ }
+ });
+ }
+ } catch (Exception e) {
html
.p()
._(
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
index 2ae495b..8fa4086 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
@@ -21,10 +21,11 @@
import static org.apache.hadoop.yarn.util.StringHelper.join;
import static org.apache.hadoop.yarn.webapp.YarnWebParams.APPLICATION_ID;
-import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
import java.util.Collection;
import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -70,10 +71,22 @@ protected void render(Block html) {
return;
}
+ final ApplicationId appIDFinal = appID;
+ UserGroupInformation callerUGI = getCallerUGI();
ApplicationReport appReport;
try {
- appReport = appContext.getApplication(appID);
- } catch (IOException e) {
+ if (callerUGI == null) {
+ appReport = appContext.getApplication(appID);
+ } else {
+ appReport = callerUGI.doAs(
+ new PrivilegedExceptionAction () {
+ @Override
+ public ApplicationReport run() throws Exception {
+ return appContext.getApplication(appIDFinal);
+ }
+ });
+ }
+ } catch (Exception e) {
String message = "Failed to read the application " + appID + ".";
LOG.error(message, e);
html.p()._(message)._();
@@ -106,8 +119,18 @@ protected void render(Block html) {
Collection attempts;
try {
- attempts = appContext.getApplicationAttempts(appID).values();
- } catch (IOException e) {
+ if (callerUGI == null) {
+ attempts = appContext.getApplicationAttempts(appID).values();
+ } else {
+ attempts = callerUGI.doAs(
+ new PrivilegedExceptionAction> () {
+ @Override
+ public Collection run() throws Exception {
+ return appContext.getApplicationAttempts(appIDFinal).values();
+ }
+ });
+ }
+ } catch (Exception e) {
String message =
"Failed to read the attempts of the application " + appID + ".";
LOG.error(message, e);
@@ -122,14 +145,24 @@ protected void render(Block html) {
._()._().tbody();
StringBuilder attemptsTableData = new StringBuilder("[\n");
- for (ApplicationAttemptReport appAttemptReport : attempts) {
+ for (final ApplicationAttemptReport appAttemptReport : attempts) {
AppAttemptInfo appAttempt = new AppAttemptInfo(appAttemptReport);
ContainerReport containerReport;
try {
- containerReport =
- appContext.getAMContainer(appAttemptReport
+ if (callerUGI == null) {
+ containerReport = appContext.getAMContainer(appAttemptReport
.getApplicationAttemptId());
- } catch (IOException e) {
+ } else {
+ containerReport = callerUGI.doAs(
+ new PrivilegedExceptionAction () {
+ @Override
+ public ContainerReport run() throws Exception {
+ return appContext.getAMContainer(appAttemptReport
+ .getApplicationAttemptId());
+ }
+ });
+ }
+ } catch (Exception e) {
String message =
"Failed to read the AM container of the application attempt "
+ appAttemptReport.getApplicationAttemptId() + ".";
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
index d4a77a8..19ea5fe 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
@@ -23,11 +23,12 @@
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.C_PROGRESSBAR;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.C_PROGRESSBAR_VALUE;
-import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
import java.util.Collection;
import java.util.HashSet;
import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.server.api.ApplicationContext;
@@ -70,10 +71,21 @@ public void render(Block html) {
}
}
+ UserGroupInformation callerUGI = getCallerUGI();
Collection appReports;
try {
- appReports = appContext.getAllApplications().values();
- } catch (IOException e) {
+ if (callerUGI == null) {
+ appReports = appContext.getAllApplications().values();
+ } else {
+ appReports = callerUGI.doAs(
+ new PrivilegedExceptionAction> () {
+ @Override
+ public Collection run() throws Exception {
+ return appContext.getAllApplications().values();
+ }
+ });
+ }
+ } catch (Exception e) {
String message = "Failed to read the applications.";
LOG.error(message, e);
html.p()._(message)._();
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
index bda80da..2bb48a8 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
@@ -20,10 +20,11 @@
import static org.apache.hadoop.yarn.util.StringHelper.join;
import static org.apache.hadoop.yarn.webapp.YarnWebParams.CONTAINER_ID;
-import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerReport;
@@ -63,10 +64,22 @@ protected void render(Block html) {
return;
}
+ final ContainerId containerIdFinal = containerId;
+ UserGroupInformation callerUGI = getCallerUGI();
ContainerReport containerReport;
try {
- containerReport = appContext.getContainer(containerId);
- } catch (IOException e) {
+ if (callerUGI == null) {
+ containerReport = appContext.getContainer(containerId);
+ } else {
+ containerReport = callerUGI.doAs(
+ new PrivilegedExceptionAction () {
+ @Override
+ public ContainerReport run() throws Exception {
+ return appContext.getContainer(containerIdFinal);
+ }
+ });
+ }
+ } catch (Exception e) {
String message = "Failed to read the container " + containerid + ".";
LOG.error(message, e);
html.p()._(message)._();
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java
index 5a82551..ce846b2 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java
@@ -18,7 +18,7 @@
package org.apache.hadoop.yarn.server.webapp;
-import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
@@ -28,6 +28,7 @@
import javax.servlet.http.HttpServletResponse;
import javax.ws.rs.WebApplicationException;
+import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -60,6 +61,7 @@ public AppsInfo getApps(HttpServletRequest req, HttpServletResponse res,
String userQuery, String queueQuery, String count, String startedBegin,
String startedEnd, String finishBegin, String finishEnd,
Set applicationTypes) {
+ UserGroupInformation callerUGI = getUser(req);
long num = 0;
boolean checkCount = false;
boolean checkStart = false;
@@ -137,8 +139,18 @@ public AppsInfo getApps(HttpServletRequest req, HttpServletResponse res,
AppsInfo allApps = new AppsInfo();
Collection appReports = null;
try {
- appReports = appContext.getAllApplications().values();
- } catch (IOException e) {
+ if (callerUGI == null) {
+ appReports = appContext.getAllApplications().values();
+ } else {
+ appReports = callerUGI.doAs(
+ new PrivilegedExceptionAction> () {
+ @Override
+ public Collection run() throws Exception {
+ return appContext.getAllApplications().values();
+ }
+ });
+ }
+ } catch (Exception e) {
throw new WebApplicationException(e);
}
for (ApplicationReport appReport : appReports) {
@@ -193,11 +205,22 @@ public AppsInfo getApps(HttpServletRequest req, HttpServletResponse res,
public AppInfo getApp(HttpServletRequest req, HttpServletResponse res,
String appId) {
- ApplicationId id = parseApplicationId(appId);
+ UserGroupInformation callerUGI = getUser(req);
+ final ApplicationId id = parseApplicationId(appId);
ApplicationReport app = null;
try {
- app = appContext.getApplication(id);
- } catch (IOException e) {
+ if (callerUGI == null) {
+ app = appContext.getApplication(id);
+ } else {
+ app = callerUGI.doAs(
+ new PrivilegedExceptionAction () {
+ @Override
+ public ApplicationReport run() throws Exception {
+ return appContext.getApplication(id);
+ }
+ });
+ }
+ } catch (Exception e) {
throw new WebApplicationException(e);
}
if (app == null) {
@@ -208,11 +231,22 @@ public AppInfo getApp(HttpServletRequest req, HttpServletResponse res,
public AppAttemptsInfo getAppAttempts(HttpServletRequest req,
HttpServletResponse res, String appId) {
- ApplicationId id = parseApplicationId(appId);
+ UserGroupInformation callerUGI = getUser(req);
+ final ApplicationId id = parseApplicationId(appId);
Collection appAttemptReports = null;
try {
- appAttemptReports = appContext.getApplicationAttempts(id).values();
- } catch (IOException e) {
+ if (callerUGI == null) {
+ appAttemptReports = appContext.getApplicationAttempts(id).values();
+ } else {
+ appAttemptReports = callerUGI.doAs(
+ new PrivilegedExceptionAction> () {
+ @Override
+ public Collection run() throws Exception {
+ return appContext.getApplicationAttempts(id).values();
+ }
+ });
+ }
+ } catch (Exception e) {
throw new WebApplicationException(e);
}
AppAttemptsInfo appAttemptsInfo = new AppAttemptsInfo();
@@ -226,13 +260,24 @@ public AppAttemptsInfo getAppAttempts(HttpServletRequest req,
public AppAttemptInfo getAppAttempt(HttpServletRequest req,
HttpServletResponse res, String appId, String appAttemptId) {
+ UserGroupInformation callerUGI = getUser(req);
ApplicationId aid = parseApplicationId(appId);
- ApplicationAttemptId aaid = parseApplicationAttemptId(appAttemptId);
+ final ApplicationAttemptId aaid = parseApplicationAttemptId(appAttemptId);
validateIds(aid, aaid, null);
ApplicationAttemptReport appAttempt = null;
try {
- appAttempt = appContext.getApplicationAttempt(aaid);
- } catch (IOException e) {
+ if (callerUGI == null) {
+ appAttempt = appContext.getApplicationAttempt(aaid);
+ } else {
+ appAttempt = callerUGI.doAs(
+ new PrivilegedExceptionAction () {
+ @Override
+ public ApplicationAttemptReport run() throws Exception {
+ return appContext.getApplicationAttempt(aaid);
+ }
+ });
+ }
+ } catch (Exception e) {
throw new WebApplicationException(e);
}
if (appAttempt == null) {
@@ -244,13 +289,24 @@ public AppAttemptInfo getAppAttempt(HttpServletRequest req,
public ContainersInfo getContainers(HttpServletRequest req,
HttpServletResponse res, String appId, String appAttemptId) {
+ UserGroupInformation callerUGI = getUser(req);
ApplicationId aid = parseApplicationId(appId);
- ApplicationAttemptId aaid = parseApplicationAttemptId(appAttemptId);
+ final ApplicationAttemptId aaid = parseApplicationAttemptId(appAttemptId);
validateIds(aid, aaid, null);
Collection containerReports = null;
try {
- containerReports = appContext.getContainers(aaid).values();
- } catch (IOException e) {
+ if (callerUGI == null) {
+ containerReports = appContext.getContainers(aaid).values();
+ } else {
+ containerReports = callerUGI.doAs(
+ new PrivilegedExceptionAction> () {
+ @Override
+ public Collection run() throws Exception {
+ return appContext.getContainers(aaid).values();
+ }
+ });
+ }
+ } catch (Exception e) {
throw new WebApplicationException(e);
}
ContainersInfo containersInfo = new ContainersInfo();
@@ -264,14 +320,25 @@ public ContainersInfo getContainers(HttpServletRequest req,
public ContainerInfo getContainer(HttpServletRequest req,
HttpServletResponse res, String appId, String appAttemptId,
String containerId) {
+ UserGroupInformation callerUGI = getUser(req);
ApplicationId aid = parseApplicationId(appId);
ApplicationAttemptId aaid = parseApplicationAttemptId(appAttemptId);
- ContainerId cid = parseContainerId(containerId);
+ final ContainerId cid = parseContainerId(containerId);
validateIds(aid, aaid, cid);
ContainerReport container = null;
try {
- container = appContext.getContainer(cid);
- } catch (IOException e) {
+ if (callerUGI == null) {
+ container = appContext.getContainer(cid);
+ } else {
+ container = callerUGI.doAs(
+ new PrivilegedExceptionAction () {
+ @Override
+ public ContainerReport run() throws Exception {
+ return appContext.getContainer(cid);
+ }
+ });
+ }
+ } catch (Exception e) {
throw new WebApplicationException(e);
}
if (container == null) {
@@ -364,4 +431,14 @@ protected void validateIds(ApplicationId appId,
throw new NotFoundException("appAttemptId and containerId don't match");
}
}
+
+ protected static UserGroupInformation getUser(HttpServletRequest req) {
+ String remoteUser = req.getRemoteUser();
+ UserGroupInformation callerUGI = null;
+ if (remoteUser != null) {
+ callerUGI = UserGroupInformation.createRemoteUser(remoteUser);
+ }
+ return callerUGI;
+ }
+
}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
index 01d5064..2048bbd 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
@@ -26,6 +26,7 @@
import org.apache.hadoop.yarn.conf.ConfigurationProvider;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
+import org.apache.hadoop.yarn.server.resourcemanager.metrics.YarnMetricsPublisher;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
@@ -98,6 +99,10 @@ void setRMDelegationTokenSecretManager(
void setRMApplicationHistoryWriter(
RMApplicationHistoryWriter rmApplicationHistoryWriter);
+ void setYarnMetricsPublisher(YarnMetricsPublisher yarnMetricsPublisher);
+
+ YarnMetricsPublisher getYarnMetricsPublisher();
+
ConfigurationProvider getConfigurationProvider();
boolean isWorkPreservingRecoveryEnabled();
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
index f72ef30..ab13ca0 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
@@ -30,6 +30,7 @@
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
+import org.apache.hadoop.yarn.server.resourcemanager.metrics.YarnMetricsPublisher;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.NullRMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@@ -81,6 +82,7 @@
private ResourceTrackerService resourceTrackerService;
private ApplicationMasterService applicationMasterService;
private RMApplicationHistoryWriter rmApplicationHistoryWriter;
+ private YarnMetricsPublisher yarnMetricsPublisher;
private ConfigurationProvider configurationProvider;
private int epoch;
@@ -346,6 +348,17 @@ public RMApplicationHistoryWriter getRMApplicationHistoryWriter() {
}
@Override
+ public void setYarnMetricsPublisher(
+ YarnMetricsPublisher yarnMetricsPublisher) {
+ this.yarnMetricsPublisher = yarnMetricsPublisher;
+ }
+
+ @Override
+ public YarnMetricsPublisher getYarnMetricsPublisher() {
+ return yarnMetricsPublisher;
+ }
+
+ @Override
public void setRMApplicationHistoryWriter(
RMApplicationHistoryWriter rmApplicationHistoryWriter) {
this.rmApplicationHistoryWriter = rmApplicationHistoryWriter;
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index e2deaa1..c329974 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -60,6 +60,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
+import org.apache.hadoop.yarn.server.resourcemanager.metrics.YarnMetricsPublisher;
import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitor;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.NullRMStateStore;
@@ -296,6 +297,10 @@ protected RMApplicationHistoryWriter createRMApplicationHistoryWriter() {
return new RMApplicationHistoryWriter();
}
+ protected YarnMetricsPublisher createYarnMetricsPublisher() {
+ return new YarnMetricsPublisher();
+ }
+
// sanity check for configurations
protected static void validateConfigs(Configuration conf) {
// validate max-attempts
@@ -399,6 +404,10 @@ protected void serviceInit(Configuration configuration) throws Exception {
addService(rmApplicationHistoryWriter);
rmContext.setRMApplicationHistoryWriter(rmApplicationHistoryWriter);
+ YarnMetricsPublisher yarnMetricsPublisher = createYarnMetricsPublisher();
+ addService(yarnMetricsPublisher);
+ rmContext.setYarnMetricsPublisher(yarnMetricsPublisher);
+
// Register event handler for NodesListManager
nodesListManager = new NodesListManager(rmContext);
rmDispatcher.register(NodesListManagerEventType.class, nodesListManager);
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AppAttemptFinishedEvent.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AppAttemptFinishedEvent.java
new file mode 100644
index 0000000..f539744
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AppAttemptFinishedEvent.java
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.metrics;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
+
+public class AppAttemptFinishedEvent extends
+ YarnMetricsEvent {
+
+ private ApplicationAttemptId appAttemptId;
+ private String trackingUrl;
+ private String originalTrackingUrl;
+ private String diagnosticsInfo;
+ private FinalApplicationStatus appStatus;
+ private YarnApplicationAttemptState state;
+
+ public AppAttemptFinishedEvent(
+ ApplicationAttemptId appAttemptId,
+ String trackingUrl,
+ String originalTrackingUrl,
+ String diagnosticsInfo,
+ FinalApplicationStatus appStatus,
+ YarnApplicationAttemptState state,
+ long finishedTime) {
+ super(YarnMetricsEventType.APP_ATTEMPT_FINISHED, finishedTime);
+ this.appAttemptId = appAttemptId;
+ // This is the tracking URL after the application attempt is finished
+ this.trackingUrl = trackingUrl;
+ this.originalTrackingUrl = originalTrackingUrl;
+ this.diagnosticsInfo = diagnosticsInfo;
+ this.appStatus = appStatus;
+ this.state = state;
+ }
+
+ @Override
+ public int hashCode() {
+ return appAttemptId.getApplicationId().hashCode();
+ }
+
+ public ApplicationAttemptId getApplicationAttemptId() {
+ return appAttemptId;
+ }
+
+ public String getTrackingUrl() {
+ return trackingUrl;
+ }
+
+ public String getOriginalTrackingURL() {
+ return originalTrackingUrl;
+ }
+
+ public String getDiagnosticsInfo() {
+ return diagnosticsInfo;
+ }
+
+ public FinalApplicationStatus getFinalApplicationStatus() {
+ return appStatus;
+ }
+
+ public YarnApplicationAttemptState getYarnApplicationAttemptState() {
+ return state;
+ }
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AppAttemptRegisteredEvent.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AppAttemptRegisteredEvent.java
new file mode 100644
index 0000000..5fbfbc9
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AppAttemptRegisteredEvent.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.metrics;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+
+public class AppAttemptRegisteredEvent extends
+ YarnMetricsEvent {
+
+ private ApplicationAttemptId appAttemptId;
+ private String host;
+ private int rpcPort;
+ private String trackingUrl;
+ private String originalTrackingUrl;
+ private ContainerId masterContainerId;
+
+ public AppAttemptRegisteredEvent(
+ ApplicationAttemptId appAttemptId,
+ String host,
+ int rpcPort,
+ String trackingUrl,
+ String originalTrackingUrl,
+ ContainerId masterContainerId,
+ long registeredTime) {
+ super(YarnMetricsEventType.APP_ATTEMPT_REGISTERED, registeredTime);
+ this.appAttemptId = appAttemptId;
+ this.host = host;
+ this.rpcPort = rpcPort;
+ // This is the tracking URL after the application attempt is registered
+ this.trackingUrl = trackingUrl;
+ this.originalTrackingUrl = originalTrackingUrl;
+ this.masterContainerId = masterContainerId;
+ }
+
+ @Override
+ public int hashCode() {
+ return appAttemptId.getApplicationId().hashCode();
+ }
+
+ public ApplicationAttemptId getApplicationAttemptId() {
+ return appAttemptId;
+ }
+
+ public String getHost() {
+ return host;
+ }
+
+ public int getRpcPort() {
+ return rpcPort;
+ }
+
+ public String getTrackingUrl() {
+ return trackingUrl;
+ }
+
+ public String getOriginalTrackingURL() {
+ return originalTrackingUrl;
+ }
+
+ public ContainerId getMasterContainerId() {
+ return masterContainerId;
+ }
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationCreatedEvent.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationCreatedEvent.java
new file mode 100644
index 0000000..79b6d29
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationCreatedEvent.java
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.metrics;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+
+public class ApplicationCreatedEvent extends
+ YarnMetricsEvent {
+
+ private ApplicationId appId;
+ private String name;
+ private String type;
+ private String user;
+ private String queue;
+ private long submittedTime;
+
+ public ApplicationCreatedEvent(ApplicationId appId,
+ String name,
+ String type,
+ String user,
+ String queue,
+ long submittedTime,
+ long createdTime) {
+ super(YarnMetricsEventType.APP_CREATED, createdTime);
+ this.appId = appId;
+ this.name = name;
+ this.type = type;
+ this.user = user;
+ this.queue = queue;
+ this.submittedTime = submittedTime;
+ }
+
+ @Override
+ public int hashCode() {
+ return appId.hashCode();
+ }
+
+ public ApplicationId getApplicationId() {
+ return appId;
+ }
+
+ public String getApplicationName() {
+ return name;
+ }
+
+ public String getApplicationType() {
+ return type;
+ }
+
+ public String getUser() {
+ return user;
+ }
+
+ public String getQueue() {
+ return queue;
+ }
+
+ public long getSubmittedTime() {
+ return submittedTime;
+ }
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationFinishedEvent.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationFinishedEvent.java
new file mode 100644
index 0000000..a96385a
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationFinishedEvent.java
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.metrics;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+
+public class ApplicationFinishedEvent extends
+ YarnMetricsEvent {
+
+ private ApplicationId appId;;
+ private String diagnosticsInfo;
+ private FinalApplicationStatus appStatus;
+ private YarnApplicationState state;
+ private ApplicationAttemptId latestAppAttemptId;
+
+ public ApplicationFinishedEvent(
+ ApplicationId appId,
+ String diagnosticsInfo,
+ FinalApplicationStatus appStatus,
+ YarnApplicationState state,
+ ApplicationAttemptId latestAppAttemptId,
+ long finishedTime) {
+ super(YarnMetricsEventType.APP_FINISHED, finishedTime);
+ this.appId = appId;
+ this.diagnosticsInfo = diagnosticsInfo;
+ this.appStatus = appStatus;
+ this.latestAppAttemptId = latestAppAttemptId;
+ this.state = state;
+ }
+
+ @Override
+ public int hashCode() {
+ return appId.hashCode();
+ }
+
+ public ApplicationId getApplicationId() {
+ return appId;
+ }
+
+ public String getDiagnosticsInfo() {
+ return diagnosticsInfo;
+ }
+
+ public FinalApplicationStatus getFinalApplicationStatus() {
+ return appStatus;
+ }
+
+ public YarnApplicationState getYarnApplicationState() {
+ return state;
+ }
+
+ public ApplicationAttemptId getLatestApplicationAttemptId() {
+ return latestAppAttemptId;
+ }
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ContainerCreatedEvent.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ContainerCreatedEvent.java
new file mode 100644
index 0000000..833d745
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ContainerCreatedEvent.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.metrics;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+
+public class ContainerCreatedEvent extends YarnMetricsEvent {
+
+ private ContainerId containerId;
+ private Resource allocatedResource;
+ private NodeId allocatedNode;
+ private Priority allocatedPriority;
+
+ public ContainerCreatedEvent(
+ ContainerId containerId,
+ Resource allocatedResource,
+ NodeId allocatedNode,
+ Priority allocatedPriority,
+ long createdTime) {
+ super(YarnMetricsEventType.CONTAINER_CREATED, createdTime);
+ this.containerId = containerId;
+ this.allocatedResource = allocatedResource;
+ this.allocatedNode = allocatedNode;
+ this.allocatedPriority = allocatedPriority;
+ }
+
+ @Override
+ public int hashCode() {
+ return containerId.getApplicationAttemptId().getApplicationId().hashCode();
+ }
+
+ public ContainerId getContainerId() {
+ return containerId;
+ }
+
+ public Resource getAllocatedResource() {
+ return allocatedResource;
+ }
+
+ public NodeId getAllocatedNode() {
+ return allocatedNode;
+ }
+
+ public Priority getAllocatedPriority() {
+ return allocatedPriority;
+ }
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ContainerFinishedEvent.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ContainerFinishedEvent.java
new file mode 100644
index 0000000..2ecfb25
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ContainerFinishedEvent.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.metrics;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+
+public class ContainerFinishedEvent extends YarnMetricsEvent {
+
+ private ContainerId containerId;
+ private String diagnosticsInfo;
+ private int containerExitStatus;
+ private ContainerState state;
+
+ public ContainerFinishedEvent(
+ ContainerId containerId,
+ String diagnosticsInfo,
+ int containerExitStatus,
+ ContainerState state,
+ long finishedTime) {
+ super(YarnMetricsEventType.CONTAINER_FINISHED, finishedTime);
+ this.containerId = containerId;
+ this.diagnosticsInfo = diagnosticsInfo;
+ this.containerExitStatus = containerExitStatus;
+ this.state = state;
+ }
+
+ @Override
+ public int hashCode() {
+ return containerId.getApplicationAttemptId().getApplicationId().hashCode();
+ }
+
+ public ContainerId getContainerId() {
+ return containerId;
+ }
+
+ public String getDiagnosticsInfo() {
+ return diagnosticsInfo;
+ }
+
+ public int getContainerExitStatus() {
+ return containerExitStatus;
+ }
+
+ public ContainerState getContainerState() {
+ return state;
+ }
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/YarnMetricsEvent.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/YarnMetricsEvent.java
new file mode 100644
index 0000000..0300a8e
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/YarnMetricsEvent.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.metrics;
+
+import org.apache.hadoop.yarn.event.AbstractEvent;
+
+public class YarnMetricsEvent extends AbstractEvent {
+
+ public YarnMetricsEvent(YarnMetricsEventType type) {
+ super(type);
+ }
+
+ public YarnMetricsEvent(YarnMetricsEventType type, long timestamp) {
+ super(type, timestamp);
+ }
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/YarnMetricsEventType.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/YarnMetricsEventType.java
new file mode 100644
index 0000000..c319f6a
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/YarnMetricsEventType.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.metrics;
+
+
+public enum YarnMetricsEventType {
+ // app events
+ APP_CREATED,
+ APP_FINISHED,
+
+ // app attempt events
+ APP_ATTEMPT_REGISTERED,
+ APP_ATTEMPT_FINISHED,
+
+ // container events
+ CONTAINER_CREATED,
+ CONTAINER_FINISHED
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/YarnMetricsPublisher.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/YarnMetricsPublisher.java
new file mode 100644
index 0000000..30a16e9
--- /dev/null
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/YarnMetricsPublisher.java
@@ -0,0 +1,484 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.metrics;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
+import org.apache.hadoop.yarn.client.api.TimelineClient;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.Event;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
+import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants;
+import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
+import org.apache.hadoop.yarn.server.metrics.ContainerMetricsConstants;
+import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+
+@Private
+@Unstable
+public class YarnMetricsPublisher extends CompositeService {
+
+ private static final Log LOG = LogFactory
+ .getLog(YarnMetricsPublisher.class);
+ private static final int MAX_GET_TIMELINE_DELEGATION_TOKEN_ATTEMPTS = 10;
+
+ private Dispatcher dispatcher;
+ // TODO: verify TimelineClient#putEntities is thread safe
+ private TimelineClient client;
+ private boolean publishYarnMetrics;
+ private int getTimelineDelegtionTokenAttempts = 0;
+ private boolean hasReceivedTimelineDelegtionToken = false;
+
+ public YarnMetricsPublisher() {
+ super(YarnMetricsPublisher.class.getName());
+ }
+
+ @Override
+ protected synchronized void serviceInit(Configuration conf) throws Exception {
+ publishYarnMetrics =
+ conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
+ YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED) &&
+ conf.getBoolean(YarnConfiguration.YARN_METRICS_PUBLISHER_ENABLED,
+ YarnConfiguration.DEFAULT_YARN_METRICS_PUBLISHER_ENABLED);
+
+ if (publishYarnMetrics) {
+ client = TimelineClient.createTimelineClient();
+ addIfService(client);
+
+ dispatcher = createDispatcher(conf);
+ dispatcher.register(YarnMetricsEventType.class,
+ new ForwardingEventHandler());
+ addIfService(dispatcher);
+ LOG.info("YARN metrics publishing service is enabled");
+ } else {
+ LOG.info("YARN metrics publishing service is not enabled");
+ }
+ super.serviceInit(conf);
+ }
+
+ @SuppressWarnings("unchecked")
+ public void appCreated(RMApp app, long createdTime) {
+ if (publishYarnMetrics) {
+ dispatcher.getEventHandler().handle(
+ new ApplicationCreatedEvent(
+ app.getApplicationId(),
+ app.getName(),
+ app.getApplicationType(),
+ app.getUser(),
+ app.getQueue(),
+ app.getSubmitTime(),
+ createdTime));
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ public void appFinished(RMApp app, RMAppState state, long finishedTime) {
+ if (publishYarnMetrics) {
+ dispatcher.getEventHandler().handle(
+ new ApplicationFinishedEvent(
+ app.getApplicationId(),
+ app.getDiagnostics().toString(),
+ app.getFinalApplicationStatus(),
+ RMServerUtils.createApplicationState(state),
+ app.getCurrentAppAttempt().getAppAttemptId(),
+ finishedTime));
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ public void appAttemptRegistered(RMAppAttempt appAttempt,
+ long registeredTime) {
+ if (publishYarnMetrics) {
+ dispatcher.getEventHandler().handle(
+ new AppAttemptRegisteredEvent(
+ appAttempt.getAppAttemptId(),
+ appAttempt.getHost(),
+ appAttempt.getRpcPort(),
+ appAttempt.getTrackingUrl(),
+ appAttempt.getOriginalTrackingUrl(),
+ appAttempt.getMasterContainer().getId(),
+ registeredTime));
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ public void appAttemptFinished(RMAppAttempt appAttempt,
+ RMAppAttemptState state, long finishedTime) {
+ if (publishYarnMetrics) {
+ dispatcher.getEventHandler().handle(
+ new AppAttemptFinishedEvent(
+ appAttempt.getAppAttemptId(),
+ appAttempt.getTrackingUrl(),
+ appAttempt.getOriginalTrackingUrl(),
+ appAttempt.getDiagnostics(),
+ appAttempt.getFinalApplicationStatus(),
+ RMServerUtils.createApplicationAttemptState(state),
+ finishedTime));
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ public void containerCreated(RMContainer container, long createdTime) {
+ if (publishYarnMetrics) {
+ dispatcher.getEventHandler().handle(
+ new ContainerCreatedEvent(
+ container.getContainerId(),
+ container.getAllocatedResource(),
+ container.getAllocatedNode(),
+ container.getAllocatedPriority(),
+ createdTime));
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ public void containerFinished(RMContainer container, long finishedTime) {
+ if (publishYarnMetrics) {
+ dispatcher.getEventHandler().handle(
+ new ContainerFinishedEvent(
+ container.getContainerId(),
+ container.getDiagnosticsInfo(),
+ container.getContainerExitStatus(),
+ container.getContainerState(),
+ finishedTime));
+ }
+ }
+
+ protected Dispatcher createDispatcher(Configuration conf) {
+ MultiThreadedDispatcher dispatcher =
+ new MultiThreadedDispatcher(
+ conf.getInt(
+ YarnConfiguration.YARN_METRICS_PUBLISHER_MULTI_THREADED_DISPATCHER_POOL_SIZE,
+ YarnConfiguration.DEFAULT_YARN_METRICS_PUBLISHER_MULTI_THREADED_DISPATCHER_POOL_SIZE));
+ dispatcher.setDrainEventsOnStop();
+ return dispatcher;
+ }
+
+ protected void handleYarnMetricsEvent(
+ YarnMetricsEvent event) {
+ switch (event.getType()) {
+ case APP_CREATED:
+ publishApplicationCreatedEvent((ApplicationCreatedEvent) event);
+ break;
+ case APP_FINISHED:
+ publishApplicationFinishedEvent((ApplicationFinishedEvent) event);
+ break;
+ case APP_ATTEMPT_REGISTERED:
+ publishAppAttemptRegisteredEvent((AppAttemptRegisteredEvent) event);
+ break;
+ case APP_ATTEMPT_FINISHED:
+ publishAppAttemptFinishedEvent((AppAttemptFinishedEvent) event);
+ break;
+ case CONTAINER_CREATED:
+ publishContainerCreatedEvent((ContainerCreatedEvent) event);
+ break;
+ case CONTAINER_FINISHED:
+ publishContainerFinishedEvent((ContainerFinishedEvent) event);
+ default:
+ LOG.error("Unknown YarnMetricsEvent type: " + event.getType());
+ }
+ }
+
+ private void publishApplicationCreatedEvent(ApplicationCreatedEvent event) {
+ TimelineEntity entity =
+ createApplicationEntity(event.getApplicationId());
+ Map entityInfo = new HashMap();
+ entityInfo.put(ApplicationMetricsConstants.NAME_ENTITY_INFO,
+ event.getApplicationName());
+ entityInfo.put(ApplicationMetricsConstants.TYPE_ENTITY_INFO,
+ event.getApplicationType());
+ entityInfo.put(ApplicationMetricsConstants.USER_ENTITY_INFO,
+ event.getUser());
+ entityInfo.put(ApplicationMetricsConstants.QUEUE_ENTITY_INFO,
+ event.getQueue());
+ entityInfo.put(ApplicationMetricsConstants.SUBMITTED_TIME_ENTITY_INFO,
+ Long.toString(event.getSubmittedTime()));
+ entity.setOtherInfo(entityInfo);
+ TimelineEvent tEvent = new TimelineEvent();
+ tEvent.setEventType(
+ ApplicationMetricsConstants.CREATED_EVENT_TYPE);
+ tEvent.setTimestamp(event.getTimestamp());
+ entity.addEvent(tEvent);
+ putEntity(entity);
+ }
+
+ private void publishApplicationFinishedEvent(ApplicationFinishedEvent event) {
+ TimelineEntity entity =
+ createApplicationEntity(event.getApplicationId());
+ TimelineEvent tEvent = new TimelineEvent();
+ tEvent.setEventType(
+ ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
+ tEvent.setTimestamp(event.getTimestamp());
+ Map eventInfo = new HashMap();
+ eventInfo.put(ApplicationMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO,
+ event.getDiagnosticsInfo());
+ eventInfo.put(ApplicationMetricsConstants.FINAL_STATUS_EVENT_INFO,
+ event.getFinalApplicationStatus().toString());
+ eventInfo.put(ApplicationMetricsConstants.STATE_EVENT_INFO,
+ event.getYarnApplicationState().toString());
+ eventInfo.put(ApplicationMetricsConstants.LATEST_APP_ATTEMPT_EVENT_INFO,
+ event.getLatestApplicationAttemptId().toString());
+ tEvent.setEventInfo(eventInfo);
+ entity.addEvent(tEvent);
+ putEntity(entity);
+ }
+
+ private static TimelineEntity createApplicationEntity(
+ ApplicationId applicationId) {
+ TimelineEntity entity = new TimelineEntity();
+ entity.setEntityType(ApplicationMetricsConstants.ENTITY_TYPE);
+ entity.setEntityId(applicationId.toString());
+ return entity;
+ }
+
+ private void
+ publishAppAttemptRegisteredEvent(AppAttemptRegisteredEvent event) {
+ // TODO: link the app attempt to an app
+ TimelineEntity entity =
+ createAppAttemptEntity(event.getApplicationAttemptId());
+ TimelineEvent tEvent = new TimelineEvent();
+ tEvent.setEventType(
+ AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE);
+ tEvent.setTimestamp(event.getTimestamp());
+ Map eventInfo = new HashMap();
+ eventInfo.put(
+ AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO,
+ event.getTrackingUrl());
+ eventInfo.put(
+ AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO,
+ event.getOriginalTrackingURL());
+ eventInfo.put(AppAttemptMetricsConstants.HOST_EVENT_INFO,
+ event.getHost());
+ eventInfo.put(AppAttemptMetricsConstants.RPC_PORT_EVENT_INFO,
+ Integer.toString(event.getRpcPort()));
+ eventInfo.put(
+ AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO,
+ event.getMasterContainerId().toString());
+ tEvent.setEventInfo(eventInfo);
+ entity.addEvent(tEvent);
+ putEntity(entity);
+ }
+
+ private void publishAppAttemptFinishedEvent(AppAttemptFinishedEvent event) {
+ TimelineEntity entity =
+ createAppAttemptEntity(event.getApplicationAttemptId());
+ TimelineEvent tEvent = new TimelineEvent();
+ tEvent.setEventType(AppAttemptMetricsConstants.FINISHED_EVENT_TYPE);
+ tEvent.setTimestamp(event.getTimestamp());
+ Map eventInfo = new HashMap();
+ eventInfo.put(
+ AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO,
+ event.getTrackingUrl());
+ eventInfo.put(
+ AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO,
+ event.getOriginalTrackingURL());
+ eventInfo.put(AppAttemptMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO,
+ event.getDiagnosticsInfo());
+ eventInfo.put(AppAttemptMetricsConstants.FINAL_STATUS_EVENT_INFO,
+ event.getFinalApplicationStatus().toString());
+ eventInfo.put(AppAttemptMetricsConstants.STATE_EVENT_INFO,
+ event.getYarnApplicationAttemptState().toString());
+ tEvent.setEventInfo(eventInfo);
+ entity.addEvent(tEvent);
+ putEntity(entity);
+ }
+
+ private static TimelineEntity createAppAttemptEntity(
+ ApplicationAttemptId appAttemptId) {
+ TimelineEntity entity = new TimelineEntity();
+ entity.setEntityType(
+ AppAttemptMetricsConstants.ENTITY_TYPE);
+ entity.setEntityId(appAttemptId.toString());
+ entity.addPrimaryFilter(AppAttemptMetricsConstants.PARENT_PRIMARY_FILTER,
+ appAttemptId.getApplicationId().toString());
+ return entity;
+ }
+
+ private void publishContainerCreatedEvent(ContainerCreatedEvent event) {
+ TimelineEntity entity = createContainerEntity(event.getContainerId());
+ Map entityInfo = new HashMap();
+ entityInfo.put(ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO,
+ event.getAllocatedResource().getMemory());
+ entityInfo.put(ContainerMetricsConstants.ALLOCATED_VCORE_ENTITY_INFO,
+ event.getAllocatedResource().getVirtualCores());
+ entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO,
+ event.getAllocatedNode().getHost());
+ entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO,
+ Integer.toString(event.getAllocatedNode().getPort()));
+ entityInfo.put(ContainerMetricsConstants.ALLOCATED_PRIORITY_ENTITY_INFO,
+ Integer.toString(event.getAllocatedPriority().getPriority()));
+ entity.setOtherInfo(entityInfo);
+ TimelineEvent tEvent = new TimelineEvent();
+ tEvent.setEventType(ContainerMetricsConstants.CREATED_EVENT_TYPE);
+ tEvent.setTimestamp(event.getTimestamp());
+ entity.addEvent(tEvent);
+ putEntity(entity);
+ }
+
+ private void publishContainerFinishedEvent(ContainerFinishedEvent event) {
+ // TODO: link the container to an app attempt
+ TimelineEntity entity = createContainerEntity(event.getContainerId());
+ TimelineEvent tEvent = new TimelineEvent();
+ tEvent.setEventType(ContainerMetricsConstants.FINISHED_EVENT_TYPE);
+ tEvent.setTimestamp(event.getTimestamp());
+ Map eventInfo = new HashMap();
+ eventInfo.put(ContainerMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO,
+ event.getDiagnosticsInfo());
+ eventInfo.put(ContainerMetricsConstants.EXIT_STATUS_EVENT_INFO,
+ Integer.toString(event.getContainerExitStatus()));
+ eventInfo.put(ContainerMetricsConstants.STATE_EVENT_INFO,
+ event.getContainerState().toString());
+ tEvent.setEventInfo(eventInfo);
+ entity.addEvent(tEvent);
+ putEntity(entity);
+ }
+
+ private static TimelineEntity createContainerEntity(
+ ContainerId containerId) {
+ TimelineEntity entity = new TimelineEntity();
+ entity.setEntityType(
+ ContainerMetricsConstants.ENTITY_TYPE);
+ entity.setEntityId(containerId.toString());
+ entity.addPrimaryFilter(ContainerMetricsConstants.PARENT_PRIMARIY_FILTER,
+ containerId.getApplicationAttemptId().toString());
+ return entity;
+ }
+
+ private void putEntity(TimelineEntity entity) {
+ if (UserGroupInformation.isSecurityEnabled()) {
+ if (!hasReceivedTimelineDelegtionToken
+ && getTimelineDelegtionTokenAttempts < MAX_GET_TIMELINE_DELEGATION_TOKEN_ATTEMPTS) {
+ try {
+ Token token =
+ client.getDelegationToken(
+ UserGroupInformation.getCurrentUser().getUserName());
+ UserGroupInformation.getCurrentUser().addToken(token);
+ hasReceivedTimelineDelegtionToken = true;
+ } catch (Exception e) {
+ LOG.error("Error happens when getting timeline delegation token", e);
+ } finally {
+ ++getTimelineDelegtionTokenAttempts;
+ if (!hasReceivedTimelineDelegtionToken
+ && getTimelineDelegtionTokenAttempts == MAX_GET_TIMELINE_DELEGATION_TOKEN_ATTEMPTS) {
+ LOG.error("Run out of the attempts to get timeline delegation token. " +
+ "Use kerberos authentication only.");
+ }
+ }
+ }
+ }
+ try {
+ client.putEntities(entity);
+ } catch (Exception e) {
+ LOG.error("Error when publishing entity [" + entity.getEntityType() + ","
+ + entity.getEntityId() + "]", e);
+ }
+ }
+
+ /**
+ * EventHandler implementation which forward events to YarnMetricsPublisher.
+ * Making use of it, YarnMetricsPublisher can avoid to have a public handle
+ * method.
+ */
+ private final class ForwardingEventHandler implements
+ EventHandler {
+
+ @Override
+ public void handle(YarnMetricsEvent event) {
+ handleYarnMetricsEvent(event);
+ }
+
+ }
+
+ @SuppressWarnings({ "rawtypes", "unchecked" })
+ protected static class MultiThreadedDispatcher extends CompositeService
+ implements Dispatcher {
+
+ private List dispatchers =
+ new ArrayList();
+
+ public MultiThreadedDispatcher(int num) {
+ super(MultiThreadedDispatcher.class.getName());
+ for (int i = 0; i < num; ++i) {
+ AsyncDispatcher dispatcher = createDispatcher();
+ dispatchers.add(dispatcher);
+ addIfService(dispatcher);
+ }
+ }
+
+ @Override
+ public EventHandler getEventHandler() {
+ return new CompositEventHandler();
+ }
+
+ @Override
+ public void register(Class extends Enum> eventType, EventHandler handler) {
+ for (AsyncDispatcher dispatcher : dispatchers) {
+ dispatcher.register(eventType, handler);
+ }
+ }
+
+ public void setDrainEventsOnStop() {
+ for (AsyncDispatcher dispatcher : dispatchers) {
+ dispatcher.setDrainEventsOnStop();
+ }
+ }
+
+ private class CompositEventHandler implements EventHandler {
+
+ @Override
+ public void handle(Event event) {
+ // Use hashCode (of ApplicationId) to dispatch the event to the child
+ // dispatcher, such that all the writing events of one application will
+ // be handled by one thread, the scheduled order of the these events
+ // will be preserved
+ int index = (event.hashCode() & Integer.MAX_VALUE) % dispatchers.size();
+ dispatchers.get(index).getEventHandler().handle(event);
+ }
+
+ }
+
+ protected AsyncDispatcher createDispatcher() {
+ return new AsyncDispatcher();
+ }
+
+ }
+
+}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
index bf374b4..a1ae3ca 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
@@ -172,6 +172,12 @@ ApplicationReport createAndGetApplicationReport(String clientUserName,
String getTrackingUrl();
/**
+ * The original tracking url for the application master.
+ * @return the original tracking url for the application master.
+ */
+ String getOriginalTrackingUrl();
+
+ /**
* the diagnostics information for the application master.
* @return the diagnostics information for the application master.
*/
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index 523e6be..d6b5c5b 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -365,6 +365,7 @@ public RMAppImpl(ApplicationId applicationId, RMContext rmContext,
this.stateMachine = stateMachineFactory.make(this);
rmContext.getRMApplicationHistoryWriter().applicationStarted(this);
+ rmContext.getYarnMetricsPublisher().appCreated(this, startTime);
}
@Override
@@ -624,6 +625,20 @@ public String getTrackingUrl() {
}
@Override
+ public String getOriginalTrackingUrl() {
+ this.readLock.lock();
+
+ try {
+ if (this.currentAttempt != null) {
+ return this.currentAttempt.getOriginalTrackingUrl();
+ }
+ return null;
+ } finally {
+ this.readLock.unlock();
+ }
+ }
+
+ @Override
public StringBuilder getDiagnostics() {
this.readLock.lock();
@@ -1106,6 +1121,8 @@ public void transition(RMAppImpl app, RMAppEvent event) {
app.rmContext.getRMApplicationHistoryWriter()
.applicationFinished(app, finalState);
+ app.rmContext.getYarnMetricsPublisher()
+ .appFinished(app, finalState, app.finishTime);
};
}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
index 000227a..2e23921 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
@@ -1141,6 +1141,9 @@ public void transition(RMAppAttemptImpl appAttempt,
appAttempt.rmContext.getRMApplicationHistoryWriter()
.applicationAttemptFinished(appAttempt, finalAttemptState);
+ appAttempt.rmContext.getYarnMetricsPublisher()
+ .appAttemptFinished(
+ appAttempt, finalAttemptState, System.currentTimeMillis());
}
}
@@ -1255,6 +1258,8 @@ public void transition(RMAppAttemptImpl appAttempt,
appAttempt.rmContext.getRMApplicationHistoryWriter()
.applicationAttemptStarted(appAttempt);
+ appAttempt.rmContext.getYarnMetricsPublisher()
+ .appAttemptRegistered(appAttempt, System.currentTimeMillis());
}
}
@@ -1713,8 +1718,8 @@ public ApplicationAttemptReport createApplicationAttemptReport() {
masterContainer == null ? null : masterContainer.getId();
attemptReport = ApplicationAttemptReport.newInstance(this
.getAppAttemptId(), this.getHost(), this.getRpcPort(), this
- .getTrackingUrl(), this.getDiagnostics(), YarnApplicationAttemptState
- .valueOf(this.getState().toString()), amId);
+ .getTrackingUrl(), this.getOriginalTrackingUrl(), this.getDiagnostics(),
+ YarnApplicationAttemptState .valueOf(this.getState().toString()), amId);
} finally {
this.readLock.unlock();
}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
index d79a0b7..8d1a076 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
@@ -186,6 +186,8 @@ public RMContainerImpl(Container container,
this.writeLock = lock.writeLock();
rmContext.getRMApplicationHistoryWriter().containerStarted(this);
+ rmContext.getYarnMetricsPublisher().containerCreated(
+ this, this.creationTime);
}
@Override
@@ -469,6 +471,8 @@ public void transition(RMContainerImpl container, RMContainerEvent event) {
container.rmContext.getRMApplicationHistoryWriter().containerFinished(
container);
+ container.rmContext.getYarnMetricsPublisher().containerFinished(
+ container, container.finishTime);
}
private static void updateMetricsIfPreempted(RMContainerImpl container) {
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
index ff60fcd..49fd9f0 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
@@ -116,6 +116,10 @@ public String getTrackingUrl() {
throw new UnsupportedOperationException("Not supported yet.");
}
@Override
+ public String getOriginalTrackingUrl() {
+ throw new UnsupportedOperationException("Not supported yet.");
+ }
+ @Override
public int getMaxAppAttempts() {
throw new UnsupportedOperationException("Not supported yet.");
}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
index b63d2fe..2fff718 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
@@ -50,6 +50,7 @@
int failCount = 0;
ApplicationId id;
String url = null;
+ String oUrl = null;
StringBuilder diagnostics = new StringBuilder();
RMAppAttempt attempt;
int maxAppAttempts = 1;
@@ -184,6 +185,15 @@ public void setTrackingUrl(String url) {
}
@Override
+ public String getOriginalTrackingUrl() {
+ return oUrl;
+ }
+
+ public void setOriginalTrackingUrl(String oUrl) {
+ this.oUrl = oUrl;
+ }
+
+ @Override
public StringBuilder getDiagnostics() {
return diagnostics;
}