You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by va...@apache.org on 2017/08/30 06:13:49 UTC

[30/50] [abbrv] hadoop git commit: YARN-6820. Restrict read access to timelineservice v2 data. Contributed by Vrushali C

YARN-6820. Restrict read access to timelineservice v2 data. Contributed by Vrushali C


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d5ff965f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d5ff965f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d5ff965f

Branch: refs/heads/trunk
Commit: d5ff965fee41fed28d3b94e11e546c1eb4c78a35
Parents: 60765af
Author: Jason Lowe <jl...@yahoo-inc.com>
Authored: Fri Aug 11 13:05:05 2017 -0500
Committer: Varun Saxena <va...@apache.org>
Committed: Wed Aug 30 11:29:53 2017 +0530

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |  31 ++
 .../reader/TimelineReaderServer.java            |   5 +
 .../reader/TimelineReaderWebServicesUtils.java  |  29 +-
 ...elineReaderWhitelistAuthorizationFilter.java | 123 ++++++
 ...WhitelistAuthorizationFilterInitializer.java |  66 ++++
 ...elineReaderWhitelistAuthorizationFilter.java | 380 +++++++++++++++++++
 6 files changed, 630 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5ff965f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index de90c69..ff9632a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -2167,6 +2167,37 @@ public class YarnConfiguration extends Configuration {
       + "hbase.configuration.file";
 
   /**
+   * The name for setting that enables or disables authentication checks
+   * for reading timeline service v2 data.
+   */
+  public static final String TIMELINE_SERVICE_READ_AUTH_ENABLED =
+      TIMELINE_SERVICE_PREFIX + "read.authentication.enabled";
+
+  /**
+   * The default setting for authentication checks for reading timeline
+   * service v2 data.
+   */
+  public static final Boolean DEFAULT_TIMELINE_SERVICE_READ_AUTH_ENABLED =
+      false;
+
+  /**
+   * The name for setting that lists the users and groups who are allowed
+   * to read timeline service v2 data. It is a comma separated list of
+   * user, followed by space, then comma separated list of groups.
+   * It will allow this list of users and groups to read the data
+   * and reject everyone else.
+   */
+  public static final String TIMELINE_SERVICE_READ_ALLOWED_USERS =
+      TIMELINE_SERVICE_PREFIX + "read.allowed.users";
+
+  /**
+   * The default value for list of the users who are allowed to read
+   * timeline service v2 data.
+   */
+  public static final String DEFAULT_TIMELINE_SERVICE_READ_ALLOWED_USERS =
+      "";
+
+  /**
    * The setting that controls how long the final value of a metric of a
    * completed app is retained before merging into the flow sum. Up to this time
    * after an application is completed out-of-order values that arrive can be

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5ff965f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
index 61f2425..5c049ea 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
@@ -40,6 +40,7 @@ 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.server.timelineservice.reader.security.TimelineReaderAuthenticationFilterInitializer;
+import org.apache.hadoop.yarn.server.timelineservice.reader.security.TimelineReaderWhitelistAuthorizationFilterInitializer;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader;
 import org.apache.hadoop.yarn.server.util.timeline.TimelineServerUtils;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
@@ -158,6 +159,10 @@ public class TimelineReaderServer extends CompositeService {
       defaultInitializers.add(
           TimelineReaderAuthenticationFilterInitializer.class.getName());
     }
+
+    defaultInitializers.add(
+        TimelineReaderWhitelistAuthorizationFilterInitializer.class.getName());
+
     TimelineServerUtils.setTimelineFilters(
         conf, initializers, defaultInitializers);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5ff965f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
index cded3a1..d613eab 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.timelineservice.reader;
 
+import java.security.Principal;
 import java.util.EnumSet;
 
 import javax.servlet.http.HttpServletRequest;
@@ -31,7 +32,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Fiel
 /**
  * Set of utility methods to be used by timeline reader web services.
  */
-final class TimelineReaderWebServicesUtils {
+public final class TimelineReaderWebServicesUtils {
 
   private TimelineReaderWebServicesUtils() {
   }
@@ -248,16 +249,36 @@ final class TimelineReaderWebServicesUtils {
   }
 
   /**
-   * Get UGI from HTTP request.
+   * Get UGI based on the remote user in the HTTP request.
+   *
    * @param req HTTP request.
    * @return UGI.
    */
-  static UserGroupInformation getUser(HttpServletRequest req) {
-    String remoteUser = req.getRemoteUser();
+  public static UserGroupInformation getUser(HttpServletRequest req) {
+    return getCallerUserGroupInformation(req, false);
+  }
+
+  /**
+   * Get UGI from the HTTP request.
+   *
+   * @param hsr HTTP request.
+   * @param usePrincipal if true, use principal name else use remote user name
+   * @return UGI.
+   */
+  public static UserGroupInformation getCallerUserGroupInformation(
+      HttpServletRequest hsr, boolean usePrincipal) {
+
+    String remoteUser = hsr.getRemoteUser();
+    if (usePrincipal) {
+      Principal princ = hsr.getUserPrincipal();
+      remoteUser = princ == null ? null : princ.getName();
+    }
+
     UserGroupInformation callerUGI = null;
     if (remoteUser != null) {
       callerUGI = UserGroupInformation.createRemoteUser(remoteUser);
     }
+
     return callerUGI;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5ff965f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderWhitelistAuthorizationFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderWhitelistAuthorizationFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderWhitelistAuthorizationFilter.java
new file mode 100644
index 0000000..b22ea3f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderWhitelistAuthorizationFilter.java
@@ -0,0 +1,123 @@
+/**
+ * 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.timelineservice.reader.security;
+
+import java.io.IOException;
+
+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.ws.rs.core.Response;
+import javax.ws.rs.core.Response.Status;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.webapp.ForbiddenException;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderWebServicesUtils;
+
+/**
+ * Filter to check if a particular user is allowed to read ATSv2 data.
+ */
+
+public class TimelineReaderWhitelistAuthorizationFilter implements Filter {
+
+  public static final String EMPTY_STRING = "";
+
+  private static final Log LOG =
+      LogFactory.getLog(TimelineReaderWhitelistAuthorizationFilter.class);
+
+  private boolean isWhitelistReadAuthEnabled = false;
+
+  private AccessControlList allowedUsersAclList;
+  private AccessControlList adminAclList;
+
+  @Override
+  public void destroy() {
+    // NOTHING
+  }
+
+  @Override
+  public void doFilter(ServletRequest request, ServletResponse response,
+      FilterChain chain) throws IOException, ServletException {
+    if (isWhitelistReadAuthEnabled) {
+      UserGroupInformation callerUGI = TimelineReaderWebServicesUtils
+          .getCallerUserGroupInformation((HttpServletRequest) request, true);
+      if (callerUGI == null) {
+        String msg = "Unable to obtain user name, user not authenticated";
+        throw new AuthorizationException(msg);
+      }
+      if (!(adminAclList.isUserAllowed(callerUGI)
+          || allowedUsersAclList.isUserAllowed(callerUGI))) {
+        String userName = callerUGI.getShortUserName();
+        String msg = "User " + userName
+            + " is not allowed to read TimelineService V2 data.";
+        Response.status(Status.FORBIDDEN).entity(msg).build();
+        throw new ForbiddenException("user " + userName
+            + " is not allowed to read TimelineService V2 data");
+      }
+    }
+    if (chain != null) {
+      chain.doFilter(request, response);
+    }
+  }
+
+  @Override
+  public void init(FilterConfig conf) throws ServletException {
+    String isWhitelistReadAuthEnabledStr = conf
+        .getInitParameter(YarnConfiguration.TIMELINE_SERVICE_READ_AUTH_ENABLED);
+    if (isWhitelistReadAuthEnabledStr == null) {
+      isWhitelistReadAuthEnabled =
+          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_READ_AUTH_ENABLED;
+    } else {
+      isWhitelistReadAuthEnabled =
+          Boolean.valueOf(isWhitelistReadAuthEnabledStr);
+    }
+
+    if (isWhitelistReadAuthEnabled) {
+      String listAllowedUsers = conf.getInitParameter(
+          YarnConfiguration.TIMELINE_SERVICE_READ_ALLOWED_USERS);
+      if (StringUtils.isEmpty(listAllowedUsers)) {
+        listAllowedUsers =
+            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_READ_ALLOWED_USERS;
+      }
+      LOG.info("listAllowedUsers=" + listAllowedUsers);
+      allowedUsersAclList = new AccessControlList(listAllowedUsers);
+      LOG.info("allowedUsersAclList=" + allowedUsersAclList.getUsers());
+      // also allow admins
+      String adminAclListStr =
+          conf.getInitParameter(YarnConfiguration.YARN_ADMIN_ACL);
+      if (StringUtils.isEmpty(adminAclListStr)) {
+        adminAclListStr =
+            TimelineReaderWhitelistAuthorizationFilter.EMPTY_STRING;
+        LOG.info("adminAclList not set, hence setting it to \"\"");
+      }
+      adminAclList = new AccessControlList(adminAclListStr);
+      LOG.info("adminAclList=" + adminAclList.getUsers());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5ff965f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderWhitelistAuthorizationFilterInitializer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderWhitelistAuthorizationFilterInitializer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderWhitelistAuthorizationFilterInitializer.java
new file mode 100644
index 0000000..a970731
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderWhitelistAuthorizationFilterInitializer.java
@@ -0,0 +1,66 @@
+/**
+ * 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.timelineservice.reader.security;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.FilterContainer;
+import org.apache.hadoop.http.FilterInitializer;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+
+/**
+ * Filter initializer to initialize
+ * {@link TimelineReaderWhitelistAuthorizationFilter} for ATSv2 timeline reader
+ * with timeline service specific configurations.
+ */
+public class TimelineReaderWhitelistAuthorizationFilterInitializer
+    extends FilterInitializer {
+
+  /**
+   * Initializes {@link TimelineReaderWhitelistAuthorizationFilter}.
+   *
+   * @param container The filter container
+   * @param conf Configuration for run-time parameters
+   */
+  @Override
+  public void initFilter(FilterContainer container, Configuration conf) {
+    Map<String, String> params = new HashMap<String, String>();
+    String isWhitelistReadAuthEnabled = Boolean.toString(
+        conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_READ_AUTH_ENABLED,
+            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_READ_AUTH_ENABLED));
+    params.put(YarnConfiguration.TIMELINE_SERVICE_READ_AUTH_ENABLED,
+        isWhitelistReadAuthEnabled);
+    params.put(YarnConfiguration.TIMELINE_SERVICE_READ_ALLOWED_USERS,
+        conf.get(YarnConfiguration.TIMELINE_SERVICE_READ_ALLOWED_USERS,
+            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_READ_ALLOWED_USERS));
+
+    params.put(YarnConfiguration.YARN_ADMIN_ACL,
+        conf.get(YarnConfiguration.YARN_ADMIN_ACL,
+            // using a default of ""
+            // instead of DEFAULT_YARN_ADMIN_ACL
+            // The reason being, DEFAULT_YARN_ADMIN_ACL is set to all users
+            // and we do not wish to allow everyone by default if
+            // read auth is enabled and YARN_ADMIN_ACL is unset
+            TimelineReaderWhitelistAuthorizationFilter.EMPTY_STRING));
+    container.addGlobalFilter("Timeline Reader Whitelist Authorization Filter",
+        TimelineReaderWhitelistAuthorizationFilter.class.getName(), params);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5ff965f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWhitelistAuthorizationFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWhitelistAuthorizationFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWhitelistAuthorizationFilter.java
new file mode 100644
index 0000000..bd4f0c2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWhitelistAuthorizationFilter.java
@@ -0,0 +1,380 @@
+/**
+ * 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.timelineservice.reader;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.security.Principal;
+import java.security.PrivilegedExceptionAction;
+import java.util.Collections;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletContext;
+import javax.servlet.ServletException;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.timelineservice.reader.security.TimelineReaderWhitelistAuthorizationFilter;
+import org.apache.hadoop.yarn.webapp.ForbiddenException;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+/**
+ * Unit tests for {@link TimelineReaderWhitelistAuthorizationFilter}.
+ *
+ */
+public class TestTimelineReaderWhitelistAuthorizationFilter {
+
+  final private static String GROUP1_NAME = "group1";
+  final private static String GROUP2_NAME = "group2";
+  final private static String GROUP3_NAME = "group3";
+  final private static String[] GROUP_NAMES =
+      new String[] {GROUP1_NAME, GROUP2_NAME, GROUP3_NAME};
+
+  private static class DummyFilterConfig implements FilterConfig {
+    final private Map<String, String> map;
+
+    DummyFilterConfig(Map<String, String> map) {
+      this.map = map;
+    }
+
+    @Override
+    public String getFilterName() {
+      return "dummy";
+    }
+
+    @Override
+    public String getInitParameter(String arg0) {
+      return map.get(arg0);
+    }
+
+    @Override
+    public Enumeration<String> getInitParameterNames() {
+      return Collections.enumeration(map.keySet());
+    }
+
+    @Override
+    public ServletContext getServletContext() {
+      return null;
+    }
+  }
+
+  @Test
+  public void checkFilterAllowedUser() throws ServletException, IOException {
+    Map<String, String> map = new HashMap<String, String>();
+    map.put(YarnConfiguration.TIMELINE_SERVICE_READ_AUTH_ENABLED, "true");
+    map.put(YarnConfiguration.TIMELINE_SERVICE_READ_ALLOWED_USERS,
+        "user1,user2");
+    TimelineReaderWhitelistAuthorizationFilter f =
+        new TimelineReaderWhitelistAuthorizationFilter();
+    FilterConfig fc = new DummyFilterConfig(map);
+    f.init(fc);
+    HttpServletRequest mockHsr = Mockito.mock(HttpServletRequest.class);
+    Mockito.when(mockHsr.getUserPrincipal()).thenReturn(new Principal() {
+      @Override
+      public String getName() {
+        return "user1";
+      }
+    });
+
+    ServletResponse r = Mockito.mock(ServletResponse.class);
+    f.doFilter(mockHsr, r, null);
+  }
+
+  @Test(expected = ForbiddenException.class)
+  public void checkFilterNotAllowedUser() throws ServletException, IOException {
+    Map<String, String> map = new HashMap<String, String>();
+    map.put(YarnConfiguration.TIMELINE_SERVICE_READ_AUTH_ENABLED, "true");
+    map.put(YarnConfiguration.TIMELINE_SERVICE_READ_ALLOWED_USERS,
+        "user1,user2");
+    TimelineReaderWhitelistAuthorizationFilter f =
+        new TimelineReaderWhitelistAuthorizationFilter();
+    FilterConfig fc = new DummyFilterConfig(map);
+    f.init(fc);
+    HttpServletRequest mockHsr = Mockito.mock(HttpServletRequest.class);
+    Mockito.when(mockHsr.getUserPrincipal()).thenReturn(new Principal() {
+      @Override
+      public String getName() {
+        return "testuser1";
+      }
+    });
+    ServletResponse r = Mockito.mock(ServletResponse.class);
+    f.doFilter(mockHsr, r, null);
+  }
+
+  @Test
+  public void checkFilterAllowedUserGroup()
+      throws ServletException, IOException, InterruptedException {
+    Map<String, String> map = new HashMap<String, String>();
+    map.put(YarnConfiguration.TIMELINE_SERVICE_READ_AUTH_ENABLED, "true");
+    map.put(YarnConfiguration.TIMELINE_SERVICE_READ_ALLOWED_USERS,
+        "user2 group1,group2");
+    TimelineReaderWhitelistAuthorizationFilter f =
+        new TimelineReaderWhitelistAuthorizationFilter();
+    FilterConfig fc = new DummyFilterConfig(map);
+    f.init(fc);
+    HttpServletRequest mockHsr = Mockito.mock(HttpServletRequest.class);
+    Mockito.when(mockHsr.getUserPrincipal()).thenReturn(new Principal() {
+      @Override
+      public String getName() {
+        return "user1";
+      }
+    });
+    ServletResponse r = Mockito.mock(ServletResponse.class);
+    UserGroupInformation user1 =
+        UserGroupInformation.createUserForTesting("user1", GROUP_NAMES);
+    user1.doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws Exception {
+        f.doFilter(mockHsr, r, null);
+        return null;
+      }
+    });
+  }
+
+  @Test(expected = ForbiddenException.class)
+  public void checkFilterNotAlloweGroup()
+      throws ServletException, IOException, InterruptedException {
+    Map<String, String> map = new HashMap<String, String>();
+    map.put(YarnConfiguration.TIMELINE_SERVICE_READ_AUTH_ENABLED, "true");
+    map.put(YarnConfiguration.TIMELINE_SERVICE_READ_ALLOWED_USERS,
+        " group5,group6");
+    TimelineReaderWhitelistAuthorizationFilter f =
+        new TimelineReaderWhitelistAuthorizationFilter();
+    FilterConfig fc = new DummyFilterConfig(map);
+    f.init(fc);
+    HttpServletRequest mockHsr = Mockito.mock(HttpServletRequest.class);
+    Mockito.when(mockHsr.getUserPrincipal()).thenReturn(new Principal() {
+      @Override
+      public String getName() {
+        return "user200";
+      }
+    });
+    ServletResponse r = Mockito.mock(ServletResponse.class);
+    UserGroupInformation user1 =
+        UserGroupInformation.createUserForTesting("user200", GROUP_NAMES);
+    user1.doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws Exception {
+        f.doFilter(mockHsr, r, null);
+        return null;
+      }
+    });
+  }
+
+  @Test
+  public void checkFilterAllowAdmins()
+      throws ServletException, IOException, InterruptedException {
+    // check that users in admin acl list are allowed to read
+    Map<String, String> map = new HashMap<String, String>();
+    map.put(YarnConfiguration.TIMELINE_SERVICE_READ_AUTH_ENABLED, "true");
+    map.put(YarnConfiguration.TIMELINE_SERVICE_READ_ALLOWED_USERS,
+        "user3 group5,group6");
+    map.put(YarnConfiguration.YARN_ADMIN_ACL, " group1,group2");
+    TimelineReaderWhitelistAuthorizationFilter f =
+        new TimelineReaderWhitelistAuthorizationFilter();
+    FilterConfig fc = new DummyFilterConfig(map);
+    f.init(fc);
+    HttpServletRequest mockHsr = Mockito.mock(HttpServletRequest.class);
+    Mockito.when(mockHsr.getUserPrincipal()).thenReturn(new Principal() {
+      @Override
+      public String getName() {
+        return "user90";
+      }
+    });
+    ServletResponse r = Mockito.mock(ServletResponse.class);
+    UserGroupInformation user1 =
+        UserGroupInformation.createUserForTesting("user90", GROUP_NAMES);
+    user1.doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws Exception {
+        f.doFilter(mockHsr, r, null);
+        return null;
+      }
+    });
+  }
+
+  @Test
+  public void checkFilterAllowAdminsWhenNoUsersSet()
+      throws ServletException, IOException, InterruptedException {
+    // check that users in admin acl list are allowed to read
+    Map<String, String> map = new HashMap<String, String>();
+    map.put(YarnConfiguration.TIMELINE_SERVICE_READ_AUTH_ENABLED, "true");
+    map.put(YarnConfiguration.YARN_ADMIN_ACL, " group1,group2");
+    TimelineReaderWhitelistAuthorizationFilter f =
+        new TimelineReaderWhitelistAuthorizationFilter();
+    FilterConfig fc = new DummyFilterConfig(map);
+    f.init(fc);
+    HttpServletRequest mockHsr = Mockito.mock(HttpServletRequest.class);
+    Mockito.when(mockHsr.getUserPrincipal()).thenReturn(new Principal() {
+      @Override
+      public String getName() {
+        return "user90";
+      }
+    });
+    ServletResponse r = Mockito.mock(ServletResponse.class);
+    UserGroupInformation user1 =
+        UserGroupInformation.createUserForTesting("user90", GROUP_NAMES);
+    user1.doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws Exception {
+        f.doFilter(mockHsr, r, null);
+        return null;
+      }
+    });
+  }
+
+  @Test(expected = ForbiddenException.class)
+  public void checkFilterAllowNoOneWhenAdminAclsEmptyAndUserAclsEmpty()
+      throws ServletException, IOException, InterruptedException {
+    // check that users in admin acl list are allowed to read
+    Map<String, String> map = new HashMap<String, String>();
+    map.put(YarnConfiguration.TIMELINE_SERVICE_READ_AUTH_ENABLED, "true");
+    TimelineReaderWhitelistAuthorizationFilter f =
+        new TimelineReaderWhitelistAuthorizationFilter();
+    FilterConfig fc = new DummyFilterConfig(map);
+    f.init(fc);
+    HttpServletRequest mockHsr = Mockito.mock(HttpServletRequest.class);
+    Mockito.when(mockHsr.getUserPrincipal()).thenReturn(new Principal() {
+      @Override
+      public String getName() {
+        return "user88";
+      }
+    });
+    ServletResponse r = Mockito.mock(ServletResponse.class);
+    UserGroupInformation user1 =
+        UserGroupInformation.createUserForTesting("user88", GROUP_NAMES);
+    user1.doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws Exception {
+        f.doFilter(mockHsr, r, null);
+        return null;
+      }
+    });
+  }
+
+  @Test
+  public void checkFilterReadAuthDisabledNoAclSettings()
+      throws ServletException, IOException, InterruptedException {
+    // Default settings for Read Auth Enabled (false)
+    // No values in admin acls or allowed read user list
+    Map<String, String> map = new HashMap<String, String>();
+    TimelineReaderWhitelistAuthorizationFilter f =
+        new TimelineReaderWhitelistAuthorizationFilter();
+    FilterConfig fc = new DummyFilterConfig(map);
+    f.init(fc);
+    HttpServletRequest mockHsr = Mockito.mock(HttpServletRequest.class);
+    Mockito.when(mockHsr.getUserPrincipal()).thenReturn(new Principal() {
+      @Override
+      public String getName() {
+        return "user437";
+      }
+    });
+    ServletResponse r = Mockito.mock(ServletResponse.class);
+    UserGroupInformation user1 =
+        UserGroupInformation.createUserForTesting("user437", GROUP_NAMES);
+    user1.doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws Exception {
+        f.doFilter(mockHsr, r, null);
+        return null;
+      }
+    });
+  }
+
+  @Test
+  public void checkFilterReadAuthDisabledButAclSettingsPopulated()
+      throws ServletException, IOException, InterruptedException {
+    Map<String, String> map = new HashMap<String, String>();
+    // Default settings for Read Auth Enabled (false)
+    // But some values in admin acls and allowed read user list
+    map.put(YarnConfiguration.YARN_ADMIN_ACL, "user1,user2 group9,group21");
+    map.put(YarnConfiguration.TIMELINE_SERVICE_READ_ALLOWED_USERS,
+        "user27,user36 group5,group6");
+    TimelineReaderWhitelistAuthorizationFilter f =
+        new TimelineReaderWhitelistAuthorizationFilter();
+    FilterConfig fc = new DummyFilterConfig(map);
+    f.init(fc);
+
+    HttpServletRequest mockHsr = mock(HttpServletRequest.class);
+    when(mockHsr.getUserPrincipal()).thenReturn(new Principal() {
+      @Override
+      public String getName() {
+        return "user37";
+      }
+    });
+
+    ServletResponse r = Mockito.mock(ServletResponse.class);
+    UserGroupInformation user1 =
+        // both username and group name are not part of admin and
+        // read allowed users
+        // but read auth is turned off
+        UserGroupInformation.createUserForTesting("user37", GROUP_NAMES);
+    user1.doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws Exception {
+        f.doFilter(mockHsr, r, null);
+        return null;
+      }
+    });
+
+    // test with username in read allowed users
+    Mockito.when(mockHsr.getUserPrincipal()).thenReturn(new Principal() {
+      @Override
+      public String getName() {
+        return "user27";
+      }
+    });
+    ServletResponse r2 = Mockito.mock(ServletResponse.class);
+    UserGroupInformation user2 =
+        UserGroupInformation.createUserForTesting("user27", GROUP_NAMES);
+    user2.doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws Exception {
+        f.doFilter(mockHsr, r2, null);
+        return null;
+      }
+    });
+
+    // test with username in admin users
+    Mockito.when(mockHsr.getUserPrincipal()).thenReturn(new Principal() {
+      @Override
+      public String getName() {
+        return "user2";
+      }
+    });
+    ServletResponse r3 = Mockito.mock(ServletResponse.class);
+    UserGroupInformation user3 =
+        UserGroupInformation.createUserForTesting("user2", GROUP_NAMES);
+    user3.doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws Exception {
+        f.doFilter(mockHsr, r3, null);
+        return null;
+      }
+    });
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org