Browse Source

YARN-2676. Enhanced Timeline auth-filter to support proxy users. Contributed by Zhijie Shen.

Vinod Kumar Vavilapalli 10 năm trước cách đây
mục cha
commit
39063cd36f
20 tập tin đã thay đổi với 492 bổ sung813 xóa
  1. 3 0
      hadoop-yarn-project/CHANGES.txt
  2. 0 269
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineAuthenticator.java
  3. 63 97
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
  4. 31 18
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/YARNDelegationTokenIdentifier.java
  5. 0 40
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineAuthenticator.java
  6. 14 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml
  7. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
  8. 4 43
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebApp.java
  9. 16 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilter.java
  10. 48 13
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
  11. 0 236
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineClientAuthenticationService.java
  12. 4 66
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelegationTokenSecretManagerService.java
  13. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryClientService.java
  14. 0 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryServer.java
  15. 215 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilter.java
  16. 81 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterInitializer.java
  17. 8 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java
  18. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServicesWithSSL.java
  19. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java
  20. 0 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -367,6 +367,9 @@ Release 2.6.0 - UNRELEASED
     YARN-1879. Marked Idempotent/AtMostOnce annotations to ApplicationMasterProtocol
     for RM fail over. (Tsuyoshi OZAWA via jianhe)
 
+    YARN-2676. Enhanced Timeline auth-filter to support proxy users. (Zhijie Shen
+    via vinodkv)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 0 - 269
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineAuthenticator.java

@@ -1,269 +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.ConnectionConfigurator;
-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 <code>KerberosAuthenticator</code> subclass that fallback to
- * {@link TimelineAuthenticationConsts}.
- */
-@Private
-@Unstable
-public class TimelineAuthenticator extends KerberosAuthenticator {
-
-  private static ObjectMapper mapper;
-  private static TimelineAuthenticator authenticator;
-  private static ConnectionConfigurator connConfigurator;
-
-  static {
-    mapper = new ObjectMapper();
-    YarnJacksonJaxbJsonProvider.configObjectMapper(mapper);
-    authenticator = new TimelineAuthenticator();
-  }
-
-  /**
-   * 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<String, String> 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 void setStaticConnectionConfigurator(
-      ConnectionConfigurator connConfigurator) {
-    TimelineAuthenticator.connConfigurator = connConfigurator;
-  }
-
-  public static Token<TimelineDelegationTokenIdentifier> getDelegationToken(
-      URL url, AuthenticatedURL.Token token, String renewer) throws IOException {
-    TimelineDelegationTokenOperation op =
-        TimelineDelegationTokenOperation.GETDELEGATIONTOKEN;
-    Map<String, String> params = new HashMap<String, String>();
-    params.put(TimelineAuthenticationConsts.OP_PARAM, op.toString());
-    params.put(TimelineAuthenticationConsts.RENEWER_PARAM, renewer);
-    url = appendParams(url, params);
-    AuthenticatedURL aUrl =
-        new AuthenticatedURL(authenticator, connConfigurator);
-    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<TimelineDelegationTokenIdentifier> dToken =
-          new Token<TimelineDelegationTokenIdentifier>();
-      dToken.decodeFromUrlString(tokenStr);
-      return dToken;
-    } catch (AuthenticationException ex) {
-      throw new IOException(ex.toString(), ex);
-    }
-  }
-
-  public static long renewDelegationToken(URL url,
-      AuthenticatedURL.Token token,
-      Token<TimelineDelegationTokenIdentifier> dToken) throws IOException {
-    Map<String, String> params = new HashMap<String, String>();
-    params.put(TimelineAuthenticationConsts.OP_PARAM,
-        TimelineDelegationTokenOperation.RENEWDELEGATIONTOKEN.toString());
-    params.put(TimelineAuthenticationConsts.TOKEN_PARAM,
-        dToken.encodeToUrlString());
-    url = appendParams(url, params);
-    AuthenticatedURL aUrl =
-        new AuthenticatedURL(authenticator, connConfigurator);
-    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<TimelineDelegationTokenIdentifier> dToken) throws IOException {
-    Map<String, String> params = new HashMap<String, String>();
-    params.put(TimelineAuthenticationConsts.OP_PARAM,
-        TimelineDelegationTokenOperation.CANCELDELEGATIONTOKEN.toString());
-    params.put(TimelineAuthenticationConsts.TOKEN_PARAM,
-        dToken.encodeToUrlString());
-    url = appendParams(url, params);
-    AuthenticatedURL aUrl =
-        new AuthenticatedURL(authenticator, connConfigurator);
-    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 <code>URL</code>.
-   * 
-   * @param url
-   *          the url.
-   * @param params
-   *          the query string parameters.
-   * 
-   * @return a <code>URL</code>
-   * 
-   * @throws IOException
-   *           thrown if an IO error occurs.
-   */
-  public static URL appendParams(URL url, Map<String, String> params)
-      throws IOException {
-    StringBuilder sb = new StringBuilder();
-    sb.append(url);
-    String separator = url.toString().contains("?") ? "&" : "?";
-    for (Map.Entry<String, String> 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 <code>HttpURLConnection</code>. 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 <code>HttpURLConnection</code>.
-   * @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()));
-      }
-    }
-  }
-
-}

+ 63 - 97
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java

@@ -20,14 +20,14 @@ package org.apache.hadoop.yarn.client.api.impl;
 
 import java.io.File;
 import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
 import java.net.HttpURLConnection;
 import java.net.URI;
 import java.net.URL;
 import java.net.URLConnection;
 import java.security.GeneralSecurityException;
+import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
 
 import javax.net.ssl.HostnameVerifier;
 import javax.net.ssl.HttpsURLConnection;
@@ -43,13 +43,14 @@ 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.authentication.client.ConnectionConfigurator;
 import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator;
+import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticator;
+import org.apache.hadoop.security.token.delegation.web.PseudoDelegationTokenAuthenticator;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomains;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
@@ -60,8 +61,6 @@ 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;
 
@@ -81,7 +80,6 @@ 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 String URL_PARAM_USER_NAME = "user.name";
   private static final Joiner JOINER = Joiner.on("");
   public final static int DEFAULT_SOCKET_TIMEOUT = 1 * 60 * 1000; // 1 minute
 
@@ -99,9 +97,11 @@ public class TimelineClientImpl extends TimelineClient {
   }
 
   private Client client;
+  private ConnectionConfigurator connConfigurator;
+  private DelegationTokenAuthenticator authenticator;
+  private DelegationTokenAuthenticatedURL.Token token;
   private URI resURI;
   private boolean isEnabled;
-  private KerberosAuthenticatedURLConnectionFactory urlFactory;
 
   public TimelineClientImpl() {
     super(TimelineClientImpl.class.getName());
@@ -116,15 +116,17 @@ public class TimelineClientImpl extends TimelineClient {
     } else {
       ClientConfig cc = new DefaultClientConfig();
       cc.getClasses().add(YarnJacksonJaxbJsonProvider.class);
-      ConnectionConfigurator connConfigurator = newConnConfigurator(conf);
+      connConfigurator = newConnConfigurator(conf);
       if (UserGroupInformation.isSecurityEnabled()) {
-        TimelineAuthenticator.setStaticConnectionConfigurator(connConfigurator);
-        urlFactory = new KerberosAuthenticatedURLConnectionFactory(connConfigurator);
-        client = new Client(new URLConnectionClientHandler(urlFactory), cc);
+        authenticator = new KerberosDelegationTokenAuthenticator();
       } else {
-        client = new Client(new URLConnectionClientHandler(
-            new PseudoAuthenticatedURLConnectionFactory(connConfigurator)), cc);
+        authenticator = new PseudoDelegationTokenAuthenticator();
       }
+      authenticator.setConnectionConfigurator(connConfigurator);
+      client = new Client(new URLConnectionClientHandler(
+          new TimelineURLConnectionFactory()), cc);
+      token = new DelegationTokenAuthenticatedURL.Token();
+
       if (YarnConfiguration.useHttps(conf)) {
         resURI = URI
             .create(JOINER.join("https://", conf.get(
@@ -137,9 +139,6 @@ public class TimelineClientImpl extends TimelineClient {
             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);
@@ -199,11 +198,34 @@ public class TimelineClientImpl extends TimelineClient {
     return resp;
   }
 
+  @SuppressWarnings("unchecked")
   @Override
   public Token<TimelineDelegationTokenIdentifier> getDelegationToken(
-      String renewer) throws IOException, YarnException {
-    return TimelineAuthenticator.getDelegationToken(resURI.toURL(),
-        urlFactory.token, renewer);
+      final String renewer) throws IOException, YarnException {
+    boolean isProxyAccess =
+        UserGroupInformation.getCurrentUser().getAuthenticationMethod()
+        == UserGroupInformation.AuthenticationMethod.PROXY;
+    UserGroupInformation callerUGI = isProxyAccess ?
+        UserGroupInformation.getCurrentUser().getRealUser()
+        : UserGroupInformation.getCurrentUser();
+    final String doAsUser = isProxyAccess ?
+        UserGroupInformation.getCurrentUser().getShortUserName() : null;
+    try {
+      return callerUGI.doAs(
+          new PrivilegedExceptionAction<Token<TimelineDelegationTokenIdentifier>>() {
+        @Override
+        public Token<TimelineDelegationTokenIdentifier> run() throws Exception {
+          DelegationTokenAuthenticatedURL authUrl =
+              new DelegationTokenAuthenticatedURL(authenticator, connConfigurator);
+          return (Token) authUrl.getDelegationToken(
+              resURI.toURL(), token, renewer, doAsUser);
+        }
+      });
+    } catch (UndeclaredThrowableException e) {
+      throw new IOException(e.getCause());
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
   }
 
   @Private
@@ -223,91 +245,35 @@ public class TimelineClientImpl extends TimelineClient {
     }
   }
 
-  private static class PseudoAuthenticatedURLConnectionFactory
-    implements HttpURLConnectionFactory {
-
-    private ConnectionConfigurator connConfigurator;
-
-    public PseudoAuthenticatedURLConnectionFactory(
-        ConnectionConfigurator connConfigurator) {
-      this.connConfigurator = connConfigurator;
-    }
-
-    @Override
-    public HttpURLConnection getHttpURLConnection(URL url) throws IOException {
-      Map<String, String> params = new HashMap<String, String>();
-      params.put(URL_PARAM_USER_NAME,
-          UserGroupInformation.getCurrentUser().getShortUserName());
-      url = TimelineAuthenticator.appendParams(url, params);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("URL with delegation token: " + url);
-      }
-      return connConfigurator.configure((HttpURLConnection) url.openConnection());
-    }
-
-  }
-  private static class KerberosAuthenticatedURLConnectionFactory
+  private class TimelineURLConnectionFactory
       implements HttpURLConnectionFactory {
 
-    private AuthenticatedURL.Token token;
-    private TimelineAuthenticator authenticator;
-    private Token<TimelineDelegationTokenIdentifier> dToken;
-    private Text service;
-    private ConnectionConfigurator connConfigurator;
-
-    public KerberosAuthenticatedURLConnectionFactory(
-        ConnectionConfigurator connConfigurator) {
-      token = new AuthenticatedURL.Token();
-      authenticator = new TimelineAuthenticator();
-      this.connConfigurator = connConfigurator;
-    }
-
     @Override
-    public HttpURLConnection getHttpURLConnection(URL url) throws IOException {
+    public HttpURLConnection getHttpURLConnection(final URL url) throws IOException {
+      boolean isProxyAccess =
+          UserGroupInformation.getCurrentUser().getAuthenticationMethod()
+          == UserGroupInformation.AuthenticationMethod.PROXY;
+      UserGroupInformation callerUGI = isProxyAccess ?
+          UserGroupInformation.getCurrentUser().getRealUser()
+          : UserGroupInformation.getCurrentUser();
+      final String doAsUser = isProxyAccess ?
+          UserGroupInformation.getCurrentUser().getShortUserName() : null;
       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());
+        return callerUGI.doAs(new PrivilegedExceptionAction<HttpURLConnection>() {
+          @Override
+          public HttpURLConnection run() throws Exception {
+            return new DelegationTokenAuthenticatedURL(
+                authenticator, connConfigurator).openConnection(url, token,
+                doAsUser);
           }
-        }
-        if (dToken != null) {
-          Map<String, String> params = new HashMap<String, String>();
-          TimelineAuthenticator.injectDelegationToken(params, dToken);
-          url = TimelineAuthenticator.appendParams(url, params);
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("URL with delegation token: " + url);
-          }
-        }
-        return new AuthenticatedURL(
-            authenticator, connConfigurator).openConnection(url, token);
-      } catch (AuthenticationException e) {
-        LOG.error("Authentication failed when openning connection [" + url
-            + "] with token [" + token + "].", e);
+        });
+      } catch (UndeclaredThrowableException e) {
+        throw new IOException(e.getCause());
+      } catch (InterruptedException e) {
         throw new IOException(e);
       }
     }
 
-    private Token<TimelineDelegationTokenIdentifier> 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;
-    }
-
   }
 
   private static ConnectionConfigurator newConnConfigurator(Configuration conf) {

+ 31 - 18
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/YARNDelegationTokenIdentifier.java

@@ -38,22 +38,9 @@ public abstract class YARNDelegationTokenIdentifier extends
   public YARNDelegationTokenIdentifier() {}
 
   public YARNDelegationTokenIdentifier(Text owner, Text renewer, Text realUser) {
-    if (owner != null) {
-      builder.setOwner(owner.toString());
-    }
-    
-    if (renewer != null) {
-      HadoopKerberosName renewerKrbName = new HadoopKerberosName(renewer.toString());
-      try {
-        builder.setRenewer(renewerKrbName.getShortName());
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    }
-    
-    if (realUser != null) {
-      builder.setRealUser(realUser.toString());
-    }
+    setOwner(owner);
+    setRenewer(renewer);
+    setRealUser(realUser);
   }
   
   /**
@@ -90,6 +77,13 @@ public abstract class YARNDelegationTokenIdentifier extends
     }
   }
 
+  @Override
+  public void setOwner(Text owner) {
+    if (builder != null && owner != null) {
+      builder.setOwner(owner.toString());
+    }
+  }
+
   public Text getRenewer() {
     String renewer = builder.getRenewer();
     if (renewer == null) {
@@ -98,7 +92,19 @@ public abstract class YARNDelegationTokenIdentifier extends
       return new Text(renewer);
     }
   }
-  
+
+  @Override
+  public void setRenewer(Text renewer) {
+    if (builder != null && renewer != null) {
+      HadoopKerberosName renewerKrbName = new HadoopKerberosName(renewer.toString());
+      try {
+        builder.setRenewer(renewerKrbName.getShortName());
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
   public Text getRealUser() {
     String realUser = builder.getRealUser();
     if (realUser == null) {
@@ -107,7 +113,14 @@ public abstract class YARNDelegationTokenIdentifier extends
       return new Text(realUser);
     }
   }
-  
+
+  @Override
+  public void setRealUser(Text realUser) {
+    if (builder != null && realUser != null) {
+      builder.setRealUser(realUser.toString());
+    }
+  }
+
   public void setIssueDate(long issueDate) {
     builder.setIssueDate(issueDate);
   }

+ 0 - 40
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineAuthenticator.java

@@ -1,40 +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.net.URL;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-public class TestTimelineAuthenticator {
-
-  @Test
-  public void testHasDelegationTokens() throws Exception {
-    TimelineAuthenticator authenticator = new TimelineAuthenticator();
-    Assert.assertFalse(authenticator.hasDelegationToken(new URL(
-        "http://localhost:8/resource")));
-    Assert.assertFalse(authenticator.hasDelegationToken(new URL(
-        "http://localhost:8/resource?other=xxxx")));
-    Assert.assertTrue(authenticator.hasDelegationToken(new URL(
-        "http://localhost:8/resource?delegation=yyyy")));
-    Assert.assertTrue(authenticator.hasDelegationToken(new URL(
-        "http://localhost:8/resource?other=xxxx&delegation=yyyy")));
-  }
-}

+ 14 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml

@@ -159,6 +159,19 @@
       <groupId>org.fusesource.leveldbjni</groupId>
       <artifactId>leveldbjni-all</artifactId>
     </dependency>
- 
+
+    <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minikdc</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-auth</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
   </dependencies>
 </project>

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java

@@ -45,6 +45,7 @@ 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.TimelineAuthenticationFilter;
 import org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilterInitializer;
 import org.apache.hadoop.yarn.server.timeline.security.TimelineDelegationTokenSecretManagerService;
 import org.apache.hadoop.yarn.server.timeline.webapp.CrossOriginFilterInitializer;
@@ -213,6 +214,8 @@ public class ApplicationHistoryServer extends CompositeService {
 
   private void startWebApp() {
     Configuration conf = getConfig();
+    TimelineAuthenticationFilter.setTimelineDelegationTokenSecretManager(
+        secretManagerService.getTimelineDelegationTokenSecretManager());
     // Always load pseudo authentication filter to parse "user.name" in an URL
     // to identify a HTTP request's user in insecure mode.
     // When Kerberos authentication type is set (i.e., secure mode is turned on),
@@ -263,15 +266,12 @@ public class ApplicationHistoryServer extends CompositeService {
                           WebAppUtils.getAHSWebAppURLWithoutScheme(conf));
     LOG.info("Instantiating AHSWebApp at " + bindAddress);
     try {
-      AHSWebApp ahsWebApp = AHSWebApp.getInstance();
-      ahsWebApp.setApplicationHistoryManager(historyManager);
-      ahsWebApp.setTimelineDelegationTokenSecretManagerService(secretManagerService);
-      ahsWebApp.setTimelineDataManager(timelineDataManager);
       webApp =
           WebApps
             .$for("applicationhistory", ApplicationHistoryClientService.class,
                 ahsClientService, "ws")
-            .with(conf).at(bindAddress).start(ahsWebApp);
+            .with(conf).at(bindAddress).start(
+                new AHSWebApp(timelineDataManager, historyManager));
     } catch (Exception e) {
       String msg = "AHSWebApp failed to start.";
       LOG.error(msg, e);

+ 4 - 43
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebApp.java

@@ -19,71 +19,34 @@ package org.apache.hadoop.yarn.server.applicationhistoryservice.webapp;
 
 import static org.apache.hadoop.yarn.util.StringHelper.pajoin;
 
-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.security.TimelineDelegationTokenSecretManagerService;
 import org.apache.hadoop.yarn.server.timeline.webapp.TimelineWebServices;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
 import org.apache.hadoop.yarn.webapp.YarnWebParams;
 
-import com.google.common.annotations.VisibleForTesting;
-
 public class AHSWebApp extends WebApp implements YarnWebParams {
 
   private ApplicationHistoryManager applicationHistoryManager;
-  private TimelineDelegationTokenSecretManagerService secretManagerService;
   private TimelineDataManager timelineDataManager;
 
-  private static AHSWebApp instance = null;
-
-  public static AHSWebApp getInstance() {
-    if (instance == null) {
-      instance = new AHSWebApp();
-    }
-    return instance;
-  }
-
-  @Private
-  @VisibleForTesting
-  public static void resetInstance() {
-    instance = null;
-  }
-
-  private AHSWebApp() {
-
-  }
-
-  public ApplicationHistoryManager getApplicationHistoryManager() {
-    return applicationHistoryManager;
-  }
-
-  public void setApplicationHistoryManager(
+  public AHSWebApp(TimelineDataManager timelineDataManager,
       ApplicationHistoryManager applicationHistoryManager) {
+    this.timelineDataManager = timelineDataManager;
     this.applicationHistoryManager = applicationHistoryManager;
   }
 
-  public TimelineDelegationTokenSecretManagerService
-      getTimelineDelegationTokenSecretManagerService() {
-    return secretManagerService;
-  }
-
-  public void setTimelineDelegationTokenSecretManagerService(
-      TimelineDelegationTokenSecretManagerService secretManagerService) {
-    this.secretManagerService = secretManagerService;
+  public ApplicationHistoryManager getApplicationHistoryManager() {
+    return applicationHistoryManager;
   }
 
   public TimelineDataManager getTimelineDataManager() {
     return timelineDataManager;
   }
 
-  public void setTimelineDataManager(TimelineDataManager timelineDataManager) {
-    this.timelineDataManager = timelineDataManager;
-  }
-
   @Override
   public void setup() {
     bind(YarnJacksonJaxbJsonProvider.class);
@@ -91,8 +54,6 @@ public class AHSWebApp extends WebApp implements YarnWebParams {
     bind(TimelineWebServices.class);
     bind(GenericExceptionHandler.class);
     bind(ApplicationContext.class).toInstance(applicationHistoryManager);
-    bind(TimelineDelegationTokenSecretManagerService.class).toInstance(
-        secretManagerService);
     bind(TimelineDataManager.class).toInstance(timelineDataManager);
     route("/", AHSController.class);
     route(pajoin("/apps", APP_STATE), AHSController.class);

+ 16 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilter.java

@@ -18,32 +18,32 @@
 
 package org.apache.hadoop.yarn.server.timeline.security;
 
-import java.util.Properties;
-
 import javax.servlet.FilterConfig;
 import javax.servlet.ServletException;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationFilter;
+import org.apache.hadoop.yarn.server.timeline.security.TimelineDelegationTokenSecretManagerService.TimelineDelegationTokenSecretManager;
 
 @Private
 @Unstable
-public class TimelineAuthenticationFilter extends AuthenticationFilter {
+public class TimelineAuthenticationFilter
+    extends DelegationTokenAuthenticationFilter {
+
+  private static TimelineDelegationTokenSecretManager secretManager;
 
   @Override
-  protected Properties getConfiguration(String configPrefix,
-      FilterConfig filterConfig) throws ServletException {
-    // In yarn-site.xml, we can simply set type to "kerberos". However, we need
-    // to replace the name here to use the customized Kerberos + DT service
-    // instead of the standard Kerberos handler.
-    Properties properties = super.getConfiguration(configPrefix, filterConfig);
-    String authType = properties.getProperty(AUTH_TYPE);
-    if (authType != null && authType.equals("kerberos")) {
-      properties.setProperty(
-          AUTH_TYPE, TimelineClientAuthenticationService.class.getName());
-    }
-    return properties;
+  public void init(FilterConfig filterConfig) throws ServletException {
+    filterConfig.getServletContext().setAttribute(
+        DelegationTokenAuthenticationFilter.DELEGATION_TOKEN_SECRET_MANAGER_ATTR,
+        secretManager);
+    super.init(filterConfig);
+  }
+
+  public static void setTimelineDelegationTokenSecretManager(
+      TimelineDelegationTokenSecretManager secretManager) {
+    TimelineAuthenticationFilter.secretManager = secretManager;
   }
 
 }

+ 48 - 13
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java

@@ -30,6 +30,16 @@ import org.apache.hadoop.http.FilterInitializer;
 import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
+import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
+import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
+import org.apache.hadoop.security.authorize.ProxyUsers;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationHandler;
+import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticationHandler;
+import org.apache.hadoop.security.token.delegation.web.PseudoDelegationTokenAuthenticationHandler;
+import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
+
+import com.google.common.annotations.VisibleForTesting;
 
 /**
  * <p>
@@ -54,6 +64,9 @@ public class TimelineAuthenticationFilterInitializer extends FilterInitializer {
   private static final String SIGNATURE_SECRET_FILE =
       TimelineAuthenticationFilter.SIGNATURE_SECRET + ".file";
 
+  @VisibleForTesting
+  Map<String, String> filterConfig;
+
   /**
    * <p>
    * Initializes {@link TimelineAuthenticationFilter}
@@ -71,14 +84,24 @@ public class TimelineAuthenticationFilterInitializer extends FilterInitializer {
    */
   @Override
   public void initFilter(FilterContainer container, Configuration conf) {
-    Map<String, String> filterConfig = new HashMap<String, String>();
+    filterConfig = new HashMap<String, String>();
 
     // setting the cookie path to root '/' so it is used for all resources.
     filterConfig.put(TimelineAuthenticationFilter.COOKIE_PATH, "/");
 
+    for (Map.Entry<String, String> entry : conf) {
+      String name = entry.getKey();
+      if (name.startsWith(ProxyUsers.CONF_HADOOP_PROXYUSER)) {
+        String value = conf.get(name);
+        name = name.substring("hadoop.".length());
+        filterConfig.put(name, value);
+      }
+    }
     for (Map.Entry<String, String> entry : conf) {
       String name = entry.getKey();
       if (name.startsWith(PREFIX)) {
+        // yarn.timeline-service.http-authentication.proxyuser will override
+        // hadoop.proxyuser
         String value = conf.get(name);
         name = name.substring(PREFIX.length());
         filterConfig.put(name, value);
@@ -108,21 +131,33 @@ public class TimelineAuthenticationFilterInitializer extends FilterInitializer {
       }
     }
 
-    // Resolve _HOST into bind address
-    String bindAddress = conf.get(HttpServer2.BIND_ADDRESS);
-    String principal =
-        filterConfig.get(TimelineClientAuthenticationService.PRINCIPAL);
-    if (principal != null) {
-      try {
-        principal = SecurityUtil.getServerPrincipal(principal, bindAddress);
-      } catch (IOException ex) {
-        throw new RuntimeException(
-            "Could not resolve Kerberos principal name: " + ex.toString(), ex);
+    String authType = filterConfig.get(AuthenticationFilter.AUTH_TYPE);
+    if (authType.equals(PseudoAuthenticationHandler.TYPE)) {
+      filterConfig.put(AuthenticationFilter.AUTH_TYPE,
+          PseudoDelegationTokenAuthenticationHandler.class.getName());
+    } else if (authType.equals(KerberosAuthenticationHandler.TYPE)) {
+      filterConfig.put(AuthenticationFilter.AUTH_TYPE,
+          KerberosDelegationTokenAuthenticationHandler.class.getName());
+
+      // Resolve _HOST into bind address
+      String bindAddress = conf.get(HttpServer2.BIND_ADDRESS);
+      String principal =
+          filterConfig.get(KerberosAuthenticationHandler.PRINCIPAL);
+      if (principal != null) {
+        try {
+          principal = SecurityUtil.getServerPrincipal(principal, bindAddress);
+        } catch (IOException ex) {
+          throw new RuntimeException(
+              "Could not resolve Kerberos principal name: " + ex.toString(), ex);
+        }
+        filterConfig.put(KerberosAuthenticationHandler.PRINCIPAL,
+            principal);
       }
-      filterConfig.put(TimelineClientAuthenticationService.PRINCIPAL,
-          principal);
     }
 
+    filterConfig.put(DelegationTokenAuthenticationHandler.TOKEN_KIND,
+        TimelineDelegationTokenIdentifier.KIND_NAME.toString());
+
     container.addGlobalFilter("Timeline Authentication Filter",
         TimelineAuthenticationFilter.class.getName(),
         filterConfig);

+ 0 - 236
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineClientAuthenticationService.java

@@ -1,236 +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.server.timeline.security;
-
-import java.io.IOException;
-import java.io.Writer;
-import java.text.MessageFormat;
-import java.util.HashSet;
-import java.util.Set;
-
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-import javax.ws.rs.core.MediaType;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authentication.client.AuthenticationException;
-import org.apache.hadoop.security.authentication.server.AuthenticationToken;
-import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.yarn.api.records.timeline.TimelineDelegationTokenResponse;
-import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
-import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenOperation;
-import org.apache.hadoop.yarn.security.client.TimelineAuthenticationConsts;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.webapp.AHSWebApp;
-import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
-import org.codehaus.jackson.map.ObjectMapper;
-
-/**
- * Server side <code>AuthenticationHandler</code> that authenticates requests
- * using the incoming delegation token as a 'delegation' query string parameter.
- * <p/>
- * If not delegation token is present in the request it delegates to the
- * {@link KerberosAuthenticationHandler}
- */
-@Private
-@Unstable
-public class TimelineClientAuthenticationService
-    extends KerberosAuthenticationHandler {
-
-  public static final String TYPE = "kerberos-dt";
-  private static final Set<String> DELEGATION_TOKEN_OPS = new HashSet<String>();
-  private static final String OP_PARAM = "op";
-  private static final String ENTER = System.getProperty("line.separator");
-
-  private ObjectMapper mapper;
-
-  static {
-    DELEGATION_TOKEN_OPS.add(
-        TimelineDelegationTokenOperation.GETDELEGATIONTOKEN.toString());
-    DELEGATION_TOKEN_OPS.add(
-        TimelineDelegationTokenOperation.RENEWDELEGATIONTOKEN.toString());
-    DELEGATION_TOKEN_OPS.add(
-        TimelineDelegationTokenOperation.CANCELDELEGATIONTOKEN.toString());
-  }
-
-  public TimelineClientAuthenticationService() {
-    super();
-    mapper = new ObjectMapper();
-    YarnJacksonJaxbJsonProvider.configObjectMapper(mapper);
-  }
-
-  /**
-   * Returns authentication type of the handler.
-   * 
-   * @return <code>delegationtoken-kerberos</code>
-   */
-  @Override
-  public String getType() {
-    return TYPE;
-  }
-
-  @Override
-  public boolean managementOperation(AuthenticationToken token,
-      HttpServletRequest request, HttpServletResponse response)
-      throws IOException, AuthenticationException {
-    boolean requestContinues = true;
-    String op = request.getParameter(OP_PARAM);
-    op = (op != null) ? op.toUpperCase() : null;
-    if (DELEGATION_TOKEN_OPS.contains(op) &&
-        !request.getMethod().equals("OPTIONS")) {
-      TimelineDelegationTokenOperation dtOp =
-          TimelineDelegationTokenOperation.valueOf(op);
-      if (dtOp.getHttpMethod().equals(request.getMethod())) {
-        if (dtOp.requiresKerberosCredentials() && token == null) {
-          response.sendError(HttpServletResponse.SC_UNAUTHORIZED,
-              MessageFormat.format(
-                  "Operation [{0}] requires SPNEGO authentication established",
-                  dtOp));
-          requestContinues = false;
-        } else {
-          TimelineDelegationTokenSecretManagerService secretManager =
-              AHSWebApp.getInstance()
-                  .getTimelineDelegationTokenSecretManagerService();
-          try {
-            TimelineDelegationTokenResponse res = null;
-            switch (dtOp) {
-              case GETDELEGATIONTOKEN:
-                UserGroupInformation ownerUGI =
-                    UserGroupInformation.createRemoteUser(token.getUserName());
-                String renewerParam =
-                    request
-                        .getParameter(TimelineAuthenticationConsts.RENEWER_PARAM);
-                if (renewerParam == null) {
-                  renewerParam = token.getUserName();
-                }
-                Token<?> dToken =
-                    secretManager.createToken(ownerUGI, renewerParam);
-                res = new TimelineDelegationTokenResponse();
-                res.setType(TimelineAuthenticationConsts.DELEGATION_TOKEN_URL);
-                res.setContent(dToken.encodeToUrlString());
-                break;
-              case RENEWDELEGATIONTOKEN:
-              case CANCELDELEGATIONTOKEN:
-                String tokenParam =
-                    request
-                        .getParameter(TimelineAuthenticationConsts.TOKEN_PARAM);
-                if (tokenParam == null) {
-                  response.sendError(HttpServletResponse.SC_BAD_REQUEST,
-                      MessageFormat
-                          .format(
-                              "Operation [{0}] requires the parameter [{1}]",
-                              dtOp,
-                              TimelineAuthenticationConsts.TOKEN_PARAM));
-                  requestContinues = false;
-                } else {
-                  if (dtOp == TimelineDelegationTokenOperation.CANCELDELEGATIONTOKEN) {
-                    Token<TimelineDelegationTokenIdentifier> dt =
-                        new Token<TimelineDelegationTokenIdentifier>();
-                    dt.decodeFromUrlString(tokenParam);
-                    secretManager.cancelToken(dt, token.getUserName());
-                  } else {
-                    Token<TimelineDelegationTokenIdentifier> dt =
-                        new Token<TimelineDelegationTokenIdentifier>();
-                    dt.decodeFromUrlString(tokenParam);
-                    long expirationTime =
-                        secretManager.renewToken(dt, token.getUserName());
-                    res = new TimelineDelegationTokenResponse();
-                    res.setType(TimelineAuthenticationConsts.DELEGATION_TOKEN_EXPIRATION_TIME);
-                    res.setContent(expirationTime);
-                  }
-                }
-                break;
-            }
-            if (requestContinues) {
-              response.setStatus(HttpServletResponse.SC_OK);
-              if (res != null) {
-                response.setContentType(MediaType.APPLICATION_JSON);
-                Writer writer = response.getWriter();
-                mapper.writeValue(writer, res);
-                writer.write(ENTER);
-                writer.flush();
-
-              }
-              requestContinues = false;
-            }
-          } catch (IOException e) {
-            throw new AuthenticationException(e.toString(), e);
-          }
-        }
-      } else {
-        response
-            .sendError(
-                HttpServletResponse.SC_BAD_REQUEST,
-                MessageFormat
-                    .format(
-                        "Wrong HTTP method [{0}] for operation [{1}], it should be [{2}]",
-                        request.getMethod(), dtOp, dtOp.getHttpMethod()));
-        requestContinues = false;
-      }
-    }
-    return requestContinues;
-  }
-
-  /**
-   * Authenticates a request looking for the <code>delegation</code>
-   * query-string parameter and verifying it is a valid token. If there is not
-   * <code>delegation</code> query-string parameter, it delegates the
-   * authentication to the {@link KerberosAuthenticationHandler} unless it is
-   * disabled.
-   * 
-   * @param request
-   *          the HTTP client request.
-   * @param response
-   *          the HTTP client response.
-   * 
-   * @return the authentication token for the authenticated request.
-   * @throws IOException
-   *           thrown if an IO error occurred.
-   * @throws AuthenticationException
-   *           thrown if the authentication failed.
-   */
-  @Override
-  public AuthenticationToken authenticate(HttpServletRequest request,
-      HttpServletResponse response)
-      throws IOException, AuthenticationException {
-    AuthenticationToken token;
-    String delegationParam =
-        request
-            .getParameter(TimelineAuthenticationConsts.DELEGATION_PARAM);
-    if (delegationParam != null) {
-      Token<TimelineDelegationTokenIdentifier> dt =
-          new Token<TimelineDelegationTokenIdentifier>();
-      dt.decodeFromUrlString(delegationParam);
-      TimelineDelegationTokenSecretManagerService secretManager =
-          AHSWebApp.getInstance()
-              .getTimelineDelegationTokenSecretManagerService();
-      UserGroupInformation ugi = secretManager.verifyToken(dt);
-      final String shortName = ugi.getShortUserName();
-      // creating a ephemeral token
-      token = new AuthenticationToken(shortName, ugi.getUserName(), getType());
-      token.setExpires(0);
-    } else {
-      token = super.authenticate(request, response);
-    }
-    return token;
-  }
-
-}

+ 4 - 66
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelegationTokenSecretManagerService.java

@@ -77,73 +77,11 @@ public class TimelineDelegationTokenSecretManagerService extends AbstractService
   }
 
   /**
-   * Creates a delegation token.
-   *
-   * @param ugi UGI creating the token.
-   * @param renewer token renewer.
-   * @return new delegation token.
-   * @throws IOException thrown if the token could not be created.
+   * Ge the instance of {link #TimelineDelegationTokenSecretManager}
+   * @return the instance of {link #TimelineDelegationTokenSecretManager}
    */
-  public Token<TimelineDelegationTokenIdentifier> createToken(
-      UserGroupInformation ugi, String renewer) throws IOException {
-    renewer = (renewer == null) ? ugi.getShortUserName() : renewer;
-    String user = ugi.getUserName();
-    Text owner = new Text(user);
-    Text realUser = null;
-    if (ugi.getRealUser() != null) {
-      realUser = new Text(ugi.getRealUser().getUserName());
-    }
-    TimelineDelegationTokenIdentifier tokenIdentifier =
-        new TimelineDelegationTokenIdentifier(owner, new Text(renewer), realUser);
-    Token<TimelineDelegationTokenIdentifier> token =
-        new Token<TimelineDelegationTokenIdentifier>(tokenIdentifier, secretManager);
-    SecurityUtil.setTokenService(token, serviceAddr);
-    return token;
-  }
-
-  /**
-   * Renews a delegation token.
-   *
-   * @param token delegation token to renew.
-   * @param renewer token renewer.
-   * @throws IOException thrown if the token could not be renewed.
-   */
-  public long renewToken(Token<TimelineDelegationTokenIdentifier> token,
-      String renewer) throws IOException {
-      return secretManager.renewToken(token, renewer);
-  }
-
-  /**
-   * Cancels a delegation token.
-   *
-   * @param token delegation token to cancel.
-   * @param canceler token canceler.
-   * @throws IOException thrown if the token could not be canceled.
-   */
-  public void cancelToken(Token<TimelineDelegationTokenIdentifier> token,
-      String canceler) throws IOException {
-    secretManager.cancelToken(token, canceler);
-  }
-
-  /**
-   * Verifies a delegation token.
-   *
-   * @param token delegation token to verify.
-   * @return the UGI for the token.
-   * @throws IOException thrown if the token could not be verified.
-   */
-  public UserGroupInformation verifyToken(Token<TimelineDelegationTokenIdentifier> token)
-    throws IOException {
-    ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
-    DataInputStream dis = new DataInputStream(buf);
-    TimelineDelegationTokenIdentifier id = new TimelineDelegationTokenIdentifier();
-    try {
-      id.readFields(dis);
-      secretManager.verifyToken(id, token.getPassword());
-    } finally {
-      dis.close();
-    }
-    return id.getUser();
+  public TimelineDelegationTokenSecretManager getTimelineDelegationTokenSecretManager() {
+    return secretManager;
   }
 
   /**

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryClientService.java

@@ -76,7 +76,6 @@ public class TestApplicationHistoryClientService extends
 
   @After
   public void tearDown() throws Exception {
-    AHSWebApp.resetInstance();
     historyServer.stop();
   }
 

+ 0 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryServer.java

@@ -105,7 +105,6 @@ public class TestApplicationHistoryServer {
       Configuration tmp = historyServer.getConfig();
       assertEquals(expectedValue, tmp.get("hadoop.http.filter.initializers"));
       historyServer.stop();
-      AHSWebApp.resetInstance();
     }
   }
 
@@ -114,6 +113,5 @@ public class TestApplicationHistoryServer {
     if (historyServer != null) {
       historyServer.stop();
     }
-    AHSWebApp.resetInstance();
   }
 }

+ 215 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilter.java

@@ -0,0 +1,215 @@
+/**
+ * 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.security;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.security.PrivilegedExceptionAction;
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.minikdc.MiniKdc;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.KerberosTestUtils;
+import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
+import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.security.token.Token;
+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.security.client.TimelineDelegationTokenIdentifier;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryServer;
+import org.apache.hadoop.yarn.server.timeline.MemoryTimelineStore;
+import org.apache.hadoop.yarn.server.timeline.TimelineStore;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestTimelineAuthenticationFilter {
+
+  private static final String FOO_USER = "foo";
+  private static final String BAR_USER = "bar";
+  private static final String HTTP_USER = "HTTP";
+
+  private static final File testRootDir = new File("target",
+      TestTimelineAuthenticationFilter.class.getName() + "-root");
+  private static File httpSpnegoKeytabFile = new File(
+      KerberosTestUtils.getKeytabFile());
+  private static String httpSpnegoPrincipal =
+      KerberosTestUtils.getServerPrincipal();
+  private static MiniKdc testMiniKDC;
+  private static ApplicationHistoryServer testTimelineServer;
+  private static Configuration conf;
+
+  @BeforeClass
+  public static void setupClass() {
+    try {
+      testMiniKDC = new MiniKdc(MiniKdc.createConf(), testRootDir);
+      testMiniKDC.start();
+      testMiniKDC.createPrincipal(
+          httpSpnegoKeytabFile, HTTP_USER + "/localhost");
+    } catch (Exception e) {
+      assertTrue("Couldn't setup MiniKDC", false);
+    }
+
+    try {
+      testTimelineServer = new ApplicationHistoryServer();
+      conf = new YarnConfiguration();
+      conf.setStrings(TimelineAuthenticationFilterInitializer.PREFIX + "type",
+          "kerberos");
+      conf.set(TimelineAuthenticationFilterInitializer.PREFIX +
+          KerberosAuthenticationHandler.PRINCIPAL, httpSpnegoPrincipal);
+      conf.set(TimelineAuthenticationFilterInitializer.PREFIX +
+          KerberosAuthenticationHandler.KEYTAB,
+          httpSpnegoKeytabFile.getAbsolutePath());
+      conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
+        "kerberos");
+      conf.set(YarnConfiguration.TIMELINE_SERVICE_PRINCIPAL,
+        httpSpnegoPrincipal);
+      conf.set(YarnConfiguration.TIMELINE_SERVICE_KEYTAB,
+        httpSpnegoKeytabFile.getAbsolutePath());
+      conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+      conf.setClass(YarnConfiguration.TIMELINE_SERVICE_STORE,
+          MemoryTimelineStore.class, TimelineStore.class);
+      conf.set(YarnConfiguration.TIMELINE_SERVICE_ADDRESS,
+          "localhost:10200");
+      conf.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
+          "localhost:8188");
+      conf.set("hadoop.proxyuser.HTTP.hosts", "*");
+      conf.set("hadoop.proxyuser.HTTP.users", FOO_USER);
+      UserGroupInformation.setConfiguration(conf);
+      testTimelineServer.init(conf);
+      testTimelineServer.start();
+    } catch (Exception e) {
+      assertTrue("Couldn't setup TimelineServer", false);
+    }
+  }
+
+  @AfterClass
+  public static void tearDownClass() {
+    if (testMiniKDC != null) {
+      testMiniKDC.stop();
+    }
+
+    if (testTimelineServer != null) {
+      testTimelineServer.stop();
+    }
+  }
+
+  private TimelineClient client;
+
+  @Before
+  public void setup() throws Exception {
+    client = TimelineClient.createTimelineClient();
+    client.init(conf);
+    client.start();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (client != null) {
+      client.stop();
+    }
+  }
+
+  @Test
+  public void testPutTimelineEntities() throws Exception {
+    KerberosTestUtils.doAs(HTTP_USER + "/localhost", new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        TimelineEntity entityToStore = new TimelineEntity();
+        entityToStore.setEntityType("TestTimelineAuthenticationFilter");
+        entityToStore.setEntityId("entity1");
+        entityToStore.setStartTime(0L);
+        TimelinePutResponse putResponse = client.putEntities(entityToStore);
+        Assert.assertEquals(0, putResponse.getErrors().size());
+        TimelineEntity entityToRead =
+            testTimelineServer.getTimelineStore().getEntity(
+                "entity1", "TestTimelineAuthenticationFilter", null);
+        Assert.assertNotNull(entityToRead);
+        return null;
+      }
+    });
+  }
+
+  @Test
+  public void testGetDelegationToken() throws Exception {
+    KerberosTestUtils.doAs(HTTP_USER + "/localhost", new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        // Let HTTP user to get the delegation for itself
+        Token<TimelineDelegationTokenIdentifier> token =
+            client.getDelegationToken(
+                UserGroupInformation.getCurrentUser().getShortUserName());
+        Assert.assertNotNull(token);
+        TimelineDelegationTokenIdentifier tDT = token.decodeIdentifier();
+        Assert.assertNotNull(tDT);
+        Assert.assertEquals(new Text(HTTP_USER), tDT.getOwner());
+
+        // Let HTTP user to get the delegation token for FOO user
+        UserGroupInformation fooUgi = UserGroupInformation.createProxyUser(
+            FOO_USER, UserGroupInformation.getCurrentUser());
+        token = fooUgi.doAs(
+            new PrivilegedExceptionAction<Token<TimelineDelegationTokenIdentifier>>() {
+          @Override
+          public Token<TimelineDelegationTokenIdentifier> run()
+              throws Exception {
+            return client.getDelegationToken(
+                UserGroupInformation.getCurrentUser().getShortUserName());
+          }
+        });
+        Assert.assertNotNull(token);
+        tDT = token.decodeIdentifier();
+        Assert.assertNotNull(tDT);
+        Assert.assertEquals(new Text(FOO_USER), tDT.getOwner());
+        Assert.assertEquals(new Text(HTTP_USER), tDT.getRealUser());
+
+        // Let HTTP user to get the delegation token for BAR user
+        UserGroupInformation barUgi = UserGroupInformation.createProxyUser(
+            BAR_USER, UserGroupInformation.getCurrentUser());
+        token = barUgi.doAs(
+            new PrivilegedExceptionAction<Token<TimelineDelegationTokenIdentifier>>() {
+          @Override
+          public Token<TimelineDelegationTokenIdentifier> run()
+              throws Exception {
+            try {
+              Token<TimelineDelegationTokenIdentifier> token =
+                  client.getDelegationToken(
+                      UserGroupInformation.getCurrentUser().getShortUserName());
+              Assert.fail();
+              return token;
+            } catch (Exception e) {
+              Assert.assertTrue(e instanceof AuthorizationException);
+              return null;
+            }
+          }
+        });
+        return null;
+      }
+    });
+  }
+
+}

+ 81 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterInitializer.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.timeline.security;
+
+import org.junit.Assert;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.FilterContainer;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+
+public class TestTimelineAuthenticationFilterInitializer {
+
+  @Test
+  public void testProxyUserConfiguration() {
+    FilterContainer container = Mockito.mock(FilterContainer.class);
+    for (int i = 0; i < 3; ++i) {
+      Configuration conf = new YarnConfiguration();
+      switch (i) {
+        case 0:
+          // hadoop.proxyuser prefix
+          conf.set("hadoop.proxyuser.foo.hosts", "*");
+          conf.set("hadoop.proxyuser.foo.users", "*");
+          conf.set("hadoop.proxyuser.foo.groups", "*");
+          break;
+        case 1:
+          // yarn.timeline-service.http-authentication.proxyuser prefix
+          conf.set("yarn.timeline-service.http-authentication.proxyuser.foo.hosts",
+              "*");
+          conf.set("yarn.timeline-service.http-authentication.proxyuser.foo.users",
+              "*");
+          conf.set("yarn.timeline-service.http-authentication.proxyuser.foo.groups",
+              "*");
+          break;
+        case 2:
+          // hadoop.proxyuser prefix has been overwritten by
+          // yarn.timeline-service.http-authentication.proxyuser prefix
+          conf.set("hadoop.proxyuser.foo.hosts", "bar");
+          conf.set("hadoop.proxyuser.foo.users", "bar");
+          conf.set("hadoop.proxyuser.foo.groups", "bar");
+          conf.set("yarn.timeline-service.http-authentication.proxyuser.foo.hosts",
+              "*");
+          conf.set("yarn.timeline-service.http-authentication.proxyuser.foo.users",
+              "*");
+          conf.set("yarn.timeline-service.http-authentication.proxyuser.foo.groups",
+              "*");
+          break;
+        default:
+          break;
+      }
+
+      TimelineAuthenticationFilterInitializer initializer =
+          new TimelineAuthenticationFilterInitializer();
+      initializer.initFilter(container, conf);
+      Assert.assertEquals(
+          "*", initializer.filterConfig.get("proxyuser.foo.hosts"));
+      Assert.assertEquals(
+          "*", initializer.filterConfig.get("proxyuser.foo.users"));
+      Assert.assertEquals(
+          "*", initializer.filterConfig.get("proxyuser.foo.groups"));
+    }
+  }
+}

+ 8 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java

@@ -42,6 +42,7 @@ import javax.ws.rs.core.Response.Status;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationHandler;
 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;
@@ -52,6 +53,7 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse.TimelinePutError;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.security.AdminACLsManager;
+import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.timeline.TestMemoryTimelineStore;
 import org.apache.hadoop.yarn.server.timeline.TimelineDataManager;
 import org.apache.hadoop.yarn.server.timeline.TimelineStore;
@@ -118,11 +120,15 @@ public class TestTimelineWebServices extends JerseyTest {
       ServletContext context = mock(ServletContext.class);
       when(filterConfig.getServletContext()).thenReturn(context);
       Enumeration<Object> names = mock(Enumeration.class);
-      when(names.hasMoreElements()).thenReturn(true, true, false);
+      when(names.hasMoreElements()).thenReturn(true, true, true, false);
       when(names.nextElement()).thenReturn(
           AuthenticationFilter.AUTH_TYPE,
-          PseudoAuthenticationHandler.ANONYMOUS_ALLOWED);
+          PseudoAuthenticationHandler.ANONYMOUS_ALLOWED,
+          DelegationTokenAuthenticationHandler.TOKEN_KIND);
       when(filterConfig.getInitParameterNames()).thenReturn(names);
+      when(filterConfig.getInitParameter(
+          DelegationTokenAuthenticationHandler.TOKEN_KIND)).thenReturn(
+              TimelineDelegationTokenIdentifier.KIND_NAME.toString());
       try {
         taFilter.init(filterConfig);
       } catch (ServletException e) {

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServicesWithSSL.java

@@ -83,7 +83,6 @@ public class TestTimelineWebServicesWithSSL {
     if (timelineServer != null) {
       timelineServer.stop();
     }
-    AHSWebApp.resetInstance();
   }
 
   @Test

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java

@@ -90,7 +90,6 @@ public class TestSystemMetricsPublisher {
     if (timelineServer != null) {
       timelineServer.stop();
     }
-    AHSWebApp.resetInstance();
   }
 
   @Test(timeout = 10000)

+ 0 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java

@@ -721,8 +721,6 @@ public class MiniYARNCluster extends CompositeService {
       if (appHistoryServer != null) {
         appHistoryServer.stop();
       }
-      AHSWebApp.resetInstance();
-      super.serviceStop();
     }
   }