Sfoglia il codice sorgente

YARN-2228. Augmented TimelineServer to load pseudo authentication filter when authentication = simple. Contributed by Zhijie Shen.
svn merge --ignore-ancestry -c 1610575 ../../trunk/


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1610576 13f79535-47bb-0310-9956-ffa450edef68

Vinod Kumar Vavilapalli 10 anni fa
parent
commit
bfefa9301f
12 ha cambiato i file con 181 aggiunte e 90 eliminazioni
  1. 3 0
      hadoop-yarn-project/CHANGES.txt
  2. 23 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
  3. 43 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/ForbiddenException.java
  4. 2 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/GenericExceptionHandler.java
  5. 18 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  6. 14 18
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
  7. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineACLsManager.java
  8. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilter.java
  9. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
  10. 8 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java
  11. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestMemoryApplicationHistoryStore.java
  12. 63 59
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java

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

@@ -25,6 +25,9 @@ Release 2.6.0 - UNRELEASED
     YARN-2274. FairScheduler: Add debug information about cluster capacity, 
     availability and reservations. (kasha)
 
+    YARN-2228. Augmented TimelineServer to load pseudo authentication filter when
+    authentication = simple. (Zhijie Shen via vinodkv)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 23 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java

@@ -72,6 +72,7 @@ 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("");
   private static Options opts;
   static {
@@ -84,17 +85,18 @@ public class TimelineClientImpl extends TimelineClient {
   private Client client;
   private URI resURI;
   private boolean isEnabled;
-  private TimelineAuthenticatedURLConnectionFactory urlFactory;
+  private KerberosAuthenticatedURLConnectionFactory urlFactory;
 
   public TimelineClientImpl() {
     super(TimelineClientImpl.class.getName());
     ClientConfig cc = new DefaultClientConfig();
     cc.getClasses().add(YarnJacksonJaxbJsonProvider.class);
     if (UserGroupInformation.isSecurityEnabled()) {
-      urlFactory = new TimelineAuthenticatedURLConnectionFactory();
+      urlFactory = new KerberosAuthenticatedURLConnectionFactory();
       client = new Client(new URLConnectionClientHandler(urlFactory), cc);
     } else {
-      client = Client.create(cc);
+      client = new Client(new URLConnectionClientHandler(
+          new PseudoAuthenticatedURLConnectionFactory()), cc);
     }
   }
 
@@ -177,7 +179,23 @@ public class TimelineClientImpl extends TimelineClient {
         .post(ClientResponse.class, entities);
   }
 
-  private static class TimelineAuthenticatedURLConnectionFactory
+  private static class PseudoAuthenticatedURLConnectionFactory
+    implements HttpURLConnectionFactory {
+
+    @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 (HttpURLConnection) url.openConnection();
+    }
+
+  }
+  private static class KerberosAuthenticatedURLConnectionFactory
       implements HttpURLConnectionFactory {
 
     private AuthenticatedURL.Token token;
@@ -185,7 +203,7 @@ public class TimelineClientImpl extends TimelineClient {
     private Token<TimelineDelegationTokenIdentifier> dToken;
     private Text service;
 
-    public TimelineAuthenticatedURLConnectionFactory() {
+    public KerberosAuthenticatedURLConnectionFactory() {
       token = new AuthenticatedURL.Token();
       authenticator = new TimelineAuthenticator();
     }

+ 43 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/ForbiddenException.java

@@ -0,0 +1,43 @@
+/**
+ * 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.webapp;
+
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Response.Status;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+
+@InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"})
+public class ForbiddenException extends WebApplicationException {
+
+  private static final long serialVersionUID = 1L;
+
+  public ForbiddenException() {
+    super(Status.FORBIDDEN);
+  }
+
+  public ForbiddenException(java.lang.Throwable cause) {
+    super(cause, Status.FORBIDDEN);
+  }
+
+  public ForbiddenException(String msg) {
+    super(new Exception(msg), Status.FORBIDDEN);
+  }
+}

+ 2 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/GenericExceptionHandler.java

@@ -81,6 +81,8 @@ public class GenericExceptionHandler implements ExceptionMapper<Exception> {
       s = Response.Status.NOT_FOUND;
     } else if (e instanceof IOException) {
       s = Response.Status.NOT_FOUND;
+    } else if (e instanceof ForbiddenException) {
+      s = Response.Status.FORBIDDEN;
     } else if (e instanceof UnsupportedOperationException) {
       s = Response.Status.BAD_REQUEST;
     } else if (e instanceof IllegalArgumentException) {

+ 18 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -1217,6 +1217,24 @@
     <value>10</value>
   </property>
 
+  <property>
+    <name>yarn.timeline-service.http-authentication.type</name>
+    <value>simple</value>
+    <description>
+      Defines authentication used for the timeline server HTTP endpoint.
+      Supported values are: simple | kerberos | #AUTHENTICATION_HANDLER_CLASSNAME#
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.timeline-service.http-authentication.simple.anonymous.allowed</name>
+    <value>true</value>
+    <description>
+      Indicates if anonymous requests are allowed by the timeline server when using
+      'simple' authentication.
+    </description>
+  </property>
+
   <property>
     <description>The Kerberos principal for the timeline server.</description>
     <name>yarn.timeline-service.principal</name>

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

@@ -28,7 +28,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.source.JvmMetrics;
 import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.service.Service;
 import org.apache.hadoop.util.ExitUtil;
@@ -178,23 +177,20 @@ public class ApplicationHistoryServer extends CompositeService {
 
   protected void startWebApp() {
     Configuration conf = getConfig();
-    // Play trick to make the customized filter will only be loaded by the
-    // timeline server when security is enabled and Kerberos authentication
-    // is used.
-    if (UserGroupInformation.isSecurityEnabled()
-        && conf
-            .get(TimelineAuthenticationFilterInitializer.PREFIX + "type", "")
-            .equals("kerberos")) {
-      String initializers = conf.get("hadoop.http.filter.initializers");
-      initializers =
-          initializers == null || initializers.length() == 0 ? "" : ","
-              + initializers;
-      if (!initializers.contains(
-          TimelineAuthenticationFilterInitializer.class.getName())) {
-        conf.set("hadoop.http.filter.initializers",
-            TimelineAuthenticationFilterInitializer.class.getName()
-            + initializers);
-      }
+    // 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),
+    // the customized filter will be loaded by the timeline server to do Kerberos
+    // + DT authentication.
+    String initializers = conf.get("hadoop.http.filter.initializers");
+    initializers =
+        initializers == null || initializers.length() == 0 ? "" : ","
+            + initializers;
+    if (!initializers.contains(
+        TimelineAuthenticationFilterInitializer.class.getName())) {
+      conf.set("hadoop.http.filter.initializers",
+          TimelineAuthenticationFilterInitializer.class.getName()
+              + initializers);
     }
     String bindAddress = WebAppUtils.getAHSWebAppURLWithoutScheme(conf);
     LOG.info("Instantiating AHSWebApp at " + bindAddress);

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineACLsManager.java

@@ -51,7 +51,8 @@ public class TimelineACLsManager {
   public boolean checkAccess(UserGroupInformation callerUGI,
       TimelineEntity entity) throws YarnException, IOException {
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Verifying the access of " + callerUGI.getShortUserName()
+      LOG.debug("Verifying the access of "
+          + (callerUGI == null ? null : callerUGI.getShortUserName())
           + " on the timeline entity "
           + new EntityIdentifier(entity.getEntityId(), entity.getEntityType()));
     }

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

@@ -38,7 +38,8 @@ public class TimelineAuthenticationFilter extends AuthenticationFilter {
     // to replace the name here to use the customized Kerberos + DT service
     // instead of the standard Kerberos handler.
     Properties properties = super.getConfiguration(configPrefix, filterConfig);
-    if (properties.getProperty(AUTH_TYPE).equals("kerberos")) {
+    String authType = properties.getProperty(AUTH_TYPE);
+    if (authType != null && authType.equals("kerberos")) {
       properties.setProperty(
           AUTH_TYPE, TimelineClientAuthenticationService.class.getName());
     }

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

@@ -47,9 +47,9 @@ import org.apache.hadoop.security.SecurityUtil;
 public class TimelineAuthenticationFilterInitializer extends FilterInitializer {
 
   /**
-   * The configuration prefix of timeline Kerberos + DT authentication
+   * The configuration prefix of timeline HTTP authentication
    */
-  public static final String PREFIX = "yarn.timeline-service.http.authentication.";
+  public static final String PREFIX = "yarn.timeline-service.http-authentication.";
 
   private static final String SIGNATURE_SECRET_FILE =
       TimelineAuthenticationFilter.SIGNATURE_SECRET + ".file";

+ 8 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java

@@ -62,11 +62,12 @@ 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.TimelineReader.Field;
+import org.apache.hadoop.yarn.server.timeline.TimelineStore;
 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.ForbiddenException;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
 
 import com.google.inject.Inject;
@@ -336,6 +337,11 @@ public class TimelineWebServices {
       return new TimelinePutResponse();
     }
     UserGroupInformation callerUGI = getUser(req);
+    if (callerUGI == null) {
+      String msg = "The owner of the posted timeline entities is not set";
+      LOG.error(msg);
+      throw new ForbiddenException(msg);
+    }
     try {
       List<EntityIdentifier> entityIDs = new ArrayList<EntityIdentifier>();
       TimelineEntities entitiesToPut = new TimelineEntities();
@@ -375,8 +381,7 @@ public class TimelineWebServices {
         // the timeline data.
         try {
           if (existingEntity == null) {
-            injectOwnerInfo(entity,
-                callerUGI == null ? "" : callerUGI.getShortUserName());
+            injectOwnerInfo(entity, callerUGI.getShortUserName());
           }
         } catch (YarnException e) {
           // Skip the entity which messes up the primary filter and record the

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

@@ -198,7 +198,7 @@ public class TestMemoryApplicationHistoryStore extends
       writeContainerFinishData(containerId);
     }
     long usedMemoryAfter = (runtime.totalMemory() - runtime.freeMemory()) / mb;
-    Assert.assertTrue((usedMemoryAfter - usedMemoryBefore) < 200);
+    Assert.assertTrue((usedMemoryAfter - usedMemoryBefore) < 400);
   }
 
 }

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

@@ -19,26 +19,26 @@
 package org.apache.hadoop.yarn.server.timeline.webapp;
 
 import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
 
-import java.io.IOException;
+import java.util.Enumeration;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import javax.inject.Singleton;
-import javax.servlet.Filter;
-import javax.servlet.FilterChain;
 import javax.servlet.FilterConfig;
 import javax.servlet.ServletException;
-import javax.servlet.ServletRequest;
-import javax.servlet.ServletResponse;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletRequestWrapper;
 import javax.ws.rs.core.MediaType;
 
 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.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
@@ -46,12 +46,11 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents;
 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.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.AdminACLsManager;
 import org.apache.hadoop.yarn.server.timeline.TestMemoryTimelineStore;
 import org.apache.hadoop.yarn.server.timeline.TimelineStore;
 import org.apache.hadoop.yarn.server.timeline.security.TimelineACLsManager;
-import org.apache.hadoop.yarn.server.timeline.webapp.TimelineWebServices.AboutInfo;
+import org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilter;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
 import org.junit.Assert;
@@ -74,11 +73,11 @@ public class TestTimelineWebServices extends JerseyTest {
   private static TimelineStore store;
   private static TimelineACLsManager timelineACLsManager;
   private static AdminACLsManager adminACLsManager;
-  private static String remoteUser;
   private long beforeTime;
 
   private Injector injector = Guice.createInjector(new ServletModule() {
 
+    @SuppressWarnings("unchecked")
     @Override
     protected void configureServlets() {
       bind(YarnJacksonJaxbJsonProvider.class);
@@ -98,7 +97,35 @@ public class TestTimelineWebServices extends JerseyTest {
       adminACLsManager = new AdminACLsManager(conf);
       bind(TimelineACLsManager.class).toInstance(timelineACLsManager);
       serve("/*").with(GuiceContainer.class);
-      filter("/*").through(TestFilter.class);
+      TimelineAuthenticationFilter taFilter = new TimelineAuthenticationFilter();
+      FilterConfig filterConfig = mock(FilterConfig.class);
+      when(filterConfig.getInitParameter(AuthenticationFilter.CONFIG_PREFIX))
+          .thenReturn(null);
+      when(filterConfig.getInitParameter(AuthenticationFilter.AUTH_TYPE))
+          .thenReturn("simple");
+      when(filterConfig.getInitParameter(
+          PseudoAuthenticationHandler.ANONYMOUS_ALLOWED)).thenReturn("true");
+      Enumeration<Object> names = mock(Enumeration.class);
+      when(names.hasMoreElements()).thenReturn(true, true, false);
+      when(names.nextElement()).thenReturn(
+          AuthenticationFilter.AUTH_TYPE,
+          PseudoAuthenticationHandler.ANONYMOUS_ALLOWED);
+      when(filterConfig.getInitParameterNames()).thenReturn(names);
+      try {
+        taFilter.init(filterConfig);
+      } catch (ServletException e) {
+        Assert.fail("Unable to initialize TimelineAuthenticationFilter: " +
+            e.getMessage());
+      }
+
+      taFilter = spy(taFilter);
+      try {
+        doNothing().when(taFilter).init(any(FilterConfig.class));
+      } catch (ServletException e) {
+        Assert.fail("Unable to initialize TimelineAuthenticationFilter: " +
+            e.getMessage());
+      }
+      filter("/*").through(taFilter);
     }
 
   });
@@ -382,6 +409,7 @@ public class TestTimelineWebServices extends JerseyTest {
     entities.addEntity(entity);
     WebResource r = resource();
     ClientResponse response = r.path("ws").path("v1").path("timeline")
+        .queryParam("user.name", "tester")
         .accept(MediaType.APPLICATION_JSON)
         .type(MediaType.APPLICATION_JSON)
         .post(ClientResponse.class, entities);
@@ -401,11 +429,21 @@ public class TestTimelineWebServices extends JerseyTest {
     entity.setStartTime(System.currentTimeMillis());
     entities.addEntity(entity);
     WebResource r = resource();
+    // No owner, will be rejected
     ClientResponse response = r.path("ws").path("v1").path("timeline")
         .accept(MediaType.APPLICATION_JSON)
         .type(MediaType.APPLICATION_JSON)
         .post(ClientResponse.class, entities);
     assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
+    assertEquals(ClientResponse.Status.FORBIDDEN,
+        response.getClientResponseStatus());
+
+    response = r.path("ws").path("v1").path("timeline")
+        .queryParam("user.name", "tester")
+        .accept(MediaType.APPLICATION_JSON)
+        .type(MediaType.APPLICATION_JSON)
+        .post(ClientResponse.class, entities);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
     TimelinePutResponse putResposne = response.getEntity(TimelinePutResponse.class);
     Assert.assertNotNull(putResposne);
     Assert.assertEquals(0, putResposne.getErrors().size());
@@ -425,7 +463,6 @@ public class TestTimelineWebServices extends JerseyTest {
   public void testPostEntitiesWithYarnACLsEnabled() throws Exception {
     AdminACLsManager oldAdminACLsManager =
         timelineACLsManager.setAdminACLsManager(adminACLsManager);
-    remoteUser = "tester";
     try {
       TimelineEntities entities = new TimelineEntities();
       TimelineEntity entity = new TimelineEntity();
@@ -435,6 +472,7 @@ public class TestTimelineWebServices extends JerseyTest {
       entities.addEntity(entity);
       WebResource r = resource();
       ClientResponse response = r.path("ws").path("v1").path("timeline")
+          .queryParam("user.name", "tester")
           .accept(MediaType.APPLICATION_JSON)
           .type(MediaType.APPLICATION_JSON)
           .post(ClientResponse.class, entities);
@@ -444,8 +482,8 @@ public class TestTimelineWebServices extends JerseyTest {
       Assert.assertEquals(0, putResponse.getErrors().size());
 
       // override/append timeline data in the same entity with different user
-      remoteUser = "other";
       response = r.path("ws").path("v1").path("timeline")
+          .queryParam("user.name", "other")
           .accept(MediaType.APPLICATION_JSON)
           .type(MediaType.APPLICATION_JSON)
           .post(ClientResponse.class, entities);
@@ -457,7 +495,6 @@ public class TestTimelineWebServices extends JerseyTest {
           putResponse.getErrors().get(0).getErrorCode());
     } finally {
       timelineACLsManager.setAdminACLsManager(oldAdminACLsManager);
-      remoteUser = null;
     }
   }
 
@@ -465,7 +502,6 @@ public class TestTimelineWebServices extends JerseyTest {
   public void testGetEntityWithYarnACLsEnabled() throws Exception {
     AdminACLsManager oldAdminACLsManager =
         timelineACLsManager.setAdminACLsManager(adminACLsManager);
-    remoteUser = "tester";
     try {
       TimelineEntities entities = new TimelineEntities();
       TimelineEntity entity = new TimelineEntity();
@@ -475,6 +511,7 @@ public class TestTimelineWebServices extends JerseyTest {
       entities.addEntity(entity);
       WebResource r = resource();
       ClientResponse response = r.path("ws").path("v1").path("timeline")
+          .queryParam("user.name", "tester")
           .accept(MediaType.APPLICATION_JSON)
           .type(MediaType.APPLICATION_JSON)
           .post(ClientResponse.class, entities);
@@ -482,6 +519,7 @@ public class TestTimelineWebServices extends JerseyTest {
       // 1. No field specification
       response = r.path("ws").path("v1").path("timeline")
           .path("test type 3").path("test id 3")
+          .queryParam("user.name", "tester")
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
@@ -492,6 +530,7 @@ public class TestTimelineWebServices extends JerseyTest {
       response = r.path("ws").path("v1").path("timeline")
           .path("test type 3").path("test id 3")
           .queryParam("fields", "relatedentities")
+          .queryParam("user.name", "tester")
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
@@ -502,6 +541,7 @@ public class TestTimelineWebServices extends JerseyTest {
       response = r.path("ws").path("v1").path("timeline")
           .path("test type 3").path("test id 3")
           .queryParam("fields", "primaryfilters")
+          .queryParam("user.name", "tester")
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
@@ -510,9 +550,9 @@ public class TestTimelineWebServices extends JerseyTest {
           TimelineStore.SystemFilter.ENTITY_OWNER.toString()));
 
       // get entity with other user
-      remoteUser = "other";
       response = r.path("ws").path("v1").path("timeline")
           .path("test type 3").path("test id 3")
+          .queryParam("user.name", "other")
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
       assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType());
@@ -520,7 +560,6 @@ public class TestTimelineWebServices extends JerseyTest {
           response.getClientResponseStatus());
     } finally {
       timelineACLsManager.setAdminACLsManager(oldAdminACLsManager);
-      remoteUser = null;
     }
   }
 
@@ -528,7 +567,6 @@ public class TestTimelineWebServices extends JerseyTest {
   public void testGetEntitiesWithYarnACLsEnabled() {
     AdminACLsManager oldAdminACLsManager =
         timelineACLsManager.setAdminACLsManager(adminACLsManager);
-    remoteUser = "tester";
     try {
       TimelineEntities entities = new TimelineEntities();
       TimelineEntity entity = new TimelineEntity();
@@ -538,11 +576,11 @@ public class TestTimelineWebServices extends JerseyTest {
       entities.addEntity(entity);
       WebResource r = resource();
       ClientResponse response = r.path("ws").path("v1").path("timeline")
+          .queryParam("user.name", "tester")
           .accept(MediaType.APPLICATION_JSON)
           .type(MediaType.APPLICATION_JSON)
           .post(ClientResponse.class, entities);
 
-      remoteUser = "other";
       entities = new TimelineEntities();
       entity = new TimelineEntity();
       entity.setEntityId("test id 5");
@@ -551,11 +589,13 @@ public class TestTimelineWebServices extends JerseyTest {
       entities.addEntity(entity);
       r = resource();
       response = r.path("ws").path("v1").path("timeline")
+          .queryParam("user.name", "other")
           .accept(MediaType.APPLICATION_JSON)
           .type(MediaType.APPLICATION_JSON)
           .post(ClientResponse.class, entities);
 
       response = r.path("ws").path("v1").path("timeline")
+          .queryParam("user.name", "other")
           .path("test type 4")
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
@@ -566,7 +606,6 @@ public class TestTimelineWebServices extends JerseyTest {
       assertEquals("test id 5", entities.getEntities().get(0).getEntityId());
     } finally {
       timelineACLsManager.setAdminACLsManager(oldAdminACLsManager);
-      remoteUser = null;
     }
   }
 
@@ -574,7 +613,6 @@ public class TestTimelineWebServices extends JerseyTest {
   public void testGetEventsWithYarnACLsEnabled() {
     AdminACLsManager oldAdminACLsManager =
         timelineACLsManager.setAdminACLsManager(adminACLsManager);
-    remoteUser = "tester";
     try {
       TimelineEntities entities = new TimelineEntities();
       TimelineEntity entity = new TimelineEntity();
@@ -588,11 +626,11 @@ public class TestTimelineWebServices extends JerseyTest {
       entities.addEntity(entity);
       WebResource r = resource();
       ClientResponse response = r.path("ws").path("v1").path("timeline")
+          .queryParam("user.name", "tester")
           .accept(MediaType.APPLICATION_JSON)
           .type(MediaType.APPLICATION_JSON)
           .post(ClientResponse.class, entities);
 
-      remoteUser = "other";
       entities = new TimelineEntities();
       entity = new TimelineEntity();
       entity.setEntityId("test id 6");
@@ -605,12 +643,14 @@ public class TestTimelineWebServices extends JerseyTest {
       entities.addEntity(entity);
       r = resource();
       response = r.path("ws").path("v1").path("timeline")
+          .queryParam("user.name", "other")
           .accept(MediaType.APPLICATION_JSON)
           .type(MediaType.APPLICATION_JSON)
           .post(ClientResponse.class, entities);
 
       response = r.path("ws").path("v1").path("timeline")
           .path("test type 5").path("events")
+          .queryParam("user.name", "other")
           .queryParam("entityId", "test id 5,test id 6")
           .accept(MediaType.APPLICATION_JSON)
           .get(ClientResponse.class);
@@ -620,43 +660,7 @@ public class TestTimelineWebServices extends JerseyTest {
       assertEquals("test id 6", events.getAllEvents().get(0).getEntityId());
     } finally {
       timelineACLsManager.setAdminACLsManager(oldAdminACLsManager);
-      remoteUser = null;
     }
   }
 
-  @Singleton
-  private static class TestFilter implements Filter {
-
-    @Override
-    public void init(FilterConfig filterConfig) throws ServletException {
-    }
-
-    @Override
-    public void doFilter(ServletRequest request, ServletResponse response,
-        FilterChain chain) throws IOException, ServletException {
-      if (request instanceof HttpServletRequest) {
-        request =
-            new TestHttpServletRequestWrapper((HttpServletRequest) request);
-      }
-      chain.doFilter(request, response);
-    }
-
-    @Override
-    public void destroy() {
-    }
-
-  }
-
-  private static class TestHttpServletRequestWrapper extends HttpServletRequestWrapper {
-
-    public TestHttpServletRequestWrapper(HttpServletRequest request) {
-      super(request);
-    }
-
-    @Override
-    public String getRemoteUser() {
-      return TestTimelineWebServices.remoteUser;
-    }
-
-  }
 }