You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@knox.apache.org by sm...@apache.org on 2022/12/22 19:15:38 UTC

[knox] branch master updated: KNOX-2839 - Identity assertion provider handles Hadoop ProxyUser auth using the 'doAs' query parameter (#681)

This is an automated email from the ASF dual-hosted git repository.

smolnar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/knox.git


The following commit(s) were added to refs/heads/master by this push:
     new 6a4d5dfb4 KNOX-2839 - Identity assertion provider handles Hadoop ProxyUser auth using the 'doAs' query parameter (#681)
6a4d5dfb4 is described below

commit 6a4d5dfb4cca531a3adda0e6330cc806d0192249
Author: Sandor Molnar <sm...@apache.org>
AuthorDate: Thu Dec 22 20:15:32 2022 +0100

    KNOX-2839 - Identity assertion provider handles Hadoop ProxyUser auth using the 'doAs' query parameter (#681)
    
    With this change, the previously added ProxyUser configuration for KnoxToken impersonation is removed becuase the same can be achieved with the appropriate identity-assertion configuration.
    
    Another very important change is that Knox's HadoopAuth filter no longer sets the 'doAs' user as primary principal. Instead, it's passed down as an ImpersonatedPrincipal and the original user (real user) remains the PrimaryPrincipal in the underlying security context (Subject).
---
 .../knox/gateway/IdentityAsserterMessages.java     |  13 +
 ...tractIdentityAsserterDeploymentContributor.java |   3 +-
 .../filter/AbstractIdentityAssertionFilter.java    |  23 +-
 .../filter/CommonIdentityAssertionFilter.java      | 139 +++++++---
 .../filter/CommonIdentityAssertionFilterTest.java  | 304 +++++++++++++++++++++
 .../filter/CommonIdentityAssertionFilterTest.java  | 155 -----------
 .../filter/ConcatIdentityAssertionFilterTest.java  |   9 +
 .../filter/HadoopGroupProviderFilterTest.java      |   6 +
 .../filter/NoImpersonationFilterTest.java          |  28 +-
 .../filter/DefaultIdentityAssertionFilterTest.java |  28 +-
 .../filter/RegexIdentityAssertionFilterTest.java   |  13 +
 .../SwitchCaseIdentityAssertionFilterTest.java     |  18 ++
 .../gateway/filter/AclsAuthorizationFilter.java    |  40 +--
 .../gateway/filter/AclsAuthorizationMessages.java  |  14 +-
 .../hadoopauth/filter/HadoopAuthFilter.java        |  17 +-
 .../hadoopauth/filter/HadoopAuthPostFilter.java    |  17 +-
 .../hadoopauth/filter/HadoopAuthFilterTest.java    |  31 ++-
 .../federation/jwt/filter/AbstractJWTFilter.java   |   9 +-
 .../jwt/filter/AccessTokenFederationFilter.java    |  10 +-
 .../filter/ShiroSubjectIdentityAdapter.java        |   5 +-
 gateway-release/home/conf/topologies/homepage.xml  |  28 +-
 .../gateway/service/knoxtoken/TokenResource.java   |  47 +---
 .../knoxtoken/TokenServiceResourceTest.java        |  28 +-
 gateway-spi/pom.xml                                |   4 +
 .../ContextAttributes.java}                        |  18 +-
 .../security/AbstractIdentityAssertionBase.java    |   2 +-
 .../apache/knox/gateway/security/SubjectUtils.java |  12 +-
 .../apache/knox/gateway/util/AuthFilterUtils.java  | 101 +++++--
 .../AuthorizationException.java}                   |  21 +-
 .../knox/gateway/util/HttpExceptionUtils.java      |  77 ++++++
 .../app/token-generation.service.ts                |  10 +-
 31 files changed, 829 insertions(+), 401 deletions(-)

diff --git a/gateway-provider-identity-assertion-common/src/main/java/org/apache/knox/gateway/IdentityAsserterMessages.java b/gateway-provider-identity-assertion-common/src/main/java/org/apache/knox/gateway/IdentityAsserterMessages.java
index 898ce8e59..6ae6ea822 100644
--- a/gateway-provider-identity-assertion-common/src/main/java/org/apache/knox/gateway/IdentityAsserterMessages.java
+++ b/gateway-provider-identity-assertion-common/src/main/java/org/apache/knox/gateway/IdentityAsserterMessages.java
@@ -22,6 +22,7 @@ import java.util.Set;
 import org.apache.knox.gateway.i18n.messages.Message;
 import org.apache.knox.gateway.i18n.messages.MessageLevel;
 import org.apache.knox.gateway.i18n.messages.Messages;
+import org.apache.knox.gateway.i18n.messages.StackTrace;
 import org.apache.knox.gateway.plang.AbstractSyntaxTree;
 import org.apache.knox.gateway.plang.SyntaxException;
 
@@ -50,4 +51,16 @@ public interface IdentityAsserterMessages {
 
   @Message( level = MessageLevel.INFO, text = "Using configured impersonation parameters: {0}")
   void impersonationConfig(String config);
+
+  @Message( level = MessageLevel.WARN, text = "Ignoring the proxyuser configuration in favor of the HadoopAuth provider's configuration.")
+  void ignoreProxyuserConfig();
+
+  @Message( level = MessageLevel.DEBUG, text = "doAsUser = {0}, RemoteUser = {1} , RemoteAddress = {2}" )
+  void hadoopAuthDoAsUser(String doAsUser, String remoteUser, String remoteAddr);
+
+  @Message( level = MessageLevel.DEBUG, text = "Proxy user Authentication successful" )
+  void hadoopAuthProxyUserSuccess();
+
+  @Message( level = MessageLevel.DEBUG, text = "Proxy user Authentication failed: {0}" )
+  void hadoopAuthProxyUserFailed(@StackTrace Throwable t);
 }
diff --git a/gateway-provider-identity-assertion-common/src/main/java/org/apache/knox/gateway/identityasserter/common/filter/AbstractIdentityAsserterDeploymentContributor.java b/gateway-provider-identity-assertion-common/src/main/java/org/apache/knox/gateway/identityasserter/common/filter/AbstractIdentityAsserterDeploymentContributor.java
index b3f5b8319..82319a815 100644
--- a/gateway-provider-identity-assertion-common/src/main/java/org/apache/knox/gateway/identityasserter/common/filter/AbstractIdentityAsserterDeploymentContributor.java
+++ b/gateway-provider-identity-assertion-common/src/main/java/org/apache/knox/gateway/identityasserter/common/filter/AbstractIdentityAsserterDeploymentContributor.java
@@ -35,10 +35,11 @@ public abstract class AbstractIdentityAsserterDeploymentContributor extends
 
   /* Service specific impersonation params that needs to be scrubbed */
   public static final String IMPERSONATION_PARAMS = "impersonation.params";
+  static final String ROLE = "identity-assertion";
 
   @Override
   public String getRole() {
-    return "identity-assertion";
+    return ROLE;
   }
 
   @Override
diff --git a/gateway-provider-identity-assertion-common/src/main/java/org/apache/knox/gateway/identityasserter/common/filter/AbstractIdentityAssertionFilter.java b/gateway-provider-identity-assertion-common/src/main/java/org/apache/knox/gateway/identityasserter/common/filter/AbstractIdentityAssertionFilter.java
index b18023b80..5202fca08 100644
--- a/gateway-provider-identity-assertion-common/src/main/java/org/apache/knox/gateway/identityasserter/common/filter/AbstractIdentityAssertionFilter.java
+++ b/gateway-provider-identity-assertion-common/src/main/java/org/apache/knox/gateway/identityasserter/common/filter/AbstractIdentityAssertionFilter.java
@@ -50,6 +50,7 @@ import org.apache.knox.gateway.i18n.resources.ResourcesFactory;
 import org.apache.knox.gateway.security.GroupPrincipal;
 import org.apache.knox.gateway.security.ImpersonatedPrincipal;
 import org.apache.knox.gateway.security.PrimaryPrincipal;
+import org.apache.knox.gateway.security.SubjectUtils;
 
 public abstract class AbstractIdentityAssertionFilter extends
   AbstractIdentityAssertionBase implements Filter {
@@ -91,7 +92,6 @@ public abstract class AbstractIdentityAssertionFilter extends
       ServletException {
         Subject subject;
         Principal impersonationPrincipal;
-        Principal primaryPrincipal;
 
         // get the current subject and determine whether we need another doAs with
         // an impersonatedPrincipal and/or mapped group principals
@@ -105,38 +105,35 @@ public abstract class AbstractIdentityAssertionFilter extends
           throw new IllegalStateException("Required Subject Missing");
         }
 
-        Set<?> currentGroups = currentSubject.getPrincipals(GroupPrincipal.class);
-
-        primaryPrincipal = (PrimaryPrincipal) currentSubject.getPrincipals(PrimaryPrincipal.class).toArray()[0];
-        if (primaryPrincipal != null) {
-          if (!primaryPrincipal.getName().equals(mappedPrincipalName)) {
+        String primaryPrincipalName = SubjectUtils.getPrimaryPrincipalName(currentSubject);
+        if (primaryPrincipalName != null) {
+          if (!primaryPrincipalName.equals(mappedPrincipalName)) {
             impersonationNeeded = true;
             AuditContext context = auditService.getContext();
             context.setProxyUsername( mappedPrincipalName );
             auditService.attachContext(context);
-            auditor.audit( Action.IDENTITY_MAPPING, primaryPrincipal.getName(),
+            auditor.audit( Action.IDENTITY_MAPPING, primaryPrincipalName,
                 ResourceType.PRINCIPAL, ActionOutcome.SUCCESS, RES.effectiveUser(mappedPrincipalName) );
           }
-        }
-        else {
+        } else {
           // something is amiss - authentication/federation providers should have run
           // before identity assertion and should have ensured that the appropriate
           // principals were added to the current subject
           // TODO: log as appropriate
-          primaryPrincipal = new PrimaryPrincipal(((HttpServletRequest) request).getUserPrincipal().getName());
+          primaryPrincipalName = ((HttpServletRequest) request).getUserPrincipal().getName();
         }
 
+        final Set<?> currentGroups = SubjectUtils.getGroupPrincipals(currentSubject);
         groupsMapped = groups != null || !currentGroups.isEmpty();
 
         if (impersonationNeeded || groupsMapped) {
           // gonna need a new subject and doAs
           subject = new Subject();
-          Set<Principal> principals = subject.getPrincipals();
-          principals.add(primaryPrincipal);
+          subject.getPrincipals().add(new PrimaryPrincipal(primaryPrincipalName));
 
           // map group principals from current Subject into newly created Subject
           for (Object obj : currentGroups) {
-            principals.add((Principal)obj);
+            subject.getPrincipals().add((Principal)obj);
           }
 
           if (impersonationNeeded) {
diff --git a/gateway-provider-identity-assertion-common/src/main/java/org/apache/knox/gateway/identityasserter/common/filter/CommonIdentityAssertionFilter.java b/gateway-provider-identity-assertion-common/src/main/java/org/apache/knox/gateway/identityasserter/common/filter/CommonIdentityAssertionFilter.java
index 15f52ccd8..537a99281 100644
--- a/gateway-provider-identity-assertion-common/src/main/java/org/apache/knox/gateway/identityasserter/common/filter/CommonIdentityAssertionFilter.java
+++ b/gateway-provider-identity-assertion-common/src/main/java/org/apache/knox/gateway/identityasserter/common/filter/CommonIdentityAssertionFilter.java
@@ -17,11 +17,14 @@
  */
 package org.apache.knox.gateway.identityasserter.common.filter;
 
+import static org.apache.knox.gateway.identityasserter.common.filter.AbstractIdentityAsserterDeploymentContributor.IMPERSONATION_PARAMS;
+import static org.apache.knox.gateway.identityasserter.common.filter.AbstractIdentityAsserterDeploymentContributor.ROLE;
+
 import java.io.IOException;
 import java.security.AccessController;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Enumeration;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -29,6 +32,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.StringTokenizer;
 import java.util.stream.Collectors;
+
 import javax.security.auth.Subject;
 import javax.servlet.FilterChain;
 import javax.servlet.FilterConfig;
@@ -38,37 +42,46 @@ import javax.servlet.ServletRequest;
 import javax.servlet.ServletResponse;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletRequestWrapper;
+import javax.servlet.http.HttpServletResponse;
 
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.knox.gateway.IdentityAsserterMessages;
+import org.apache.knox.gateway.context.ContextAttributes;
 import org.apache.knox.gateway.i18n.messages.MessagesFactory;
 import org.apache.knox.gateway.plang.AbstractSyntaxTree;
 import org.apache.knox.gateway.plang.Parser;
 import org.apache.knox.gateway.plang.SyntaxException;
 import org.apache.knox.gateway.security.GroupPrincipal;
+import org.apache.knox.gateway.security.SubjectUtils;
 import org.apache.knox.gateway.security.principal.PrincipalMappingException;
 import org.apache.knox.gateway.security.principal.SimplePrincipalMapper;
-
-import static org.apache.knox.gateway.identityasserter.common.filter.AbstractIdentityAsserterDeploymentContributor.IMPERSONATION_PARAMS;
+import org.apache.knox.gateway.services.GatewayServices;
+import org.apache.knox.gateway.util.AuthFilterUtils;
+import org.apache.knox.gateway.util.AuthorizationException;
+import org.apache.knox.gateway.util.HttpExceptionUtils;
 
 public class CommonIdentityAssertionFilter extends AbstractIdentityAssertionFilter {
-  public static final String VIRTUAL_GROUP_MAPPING_PREFIX = "group.mapping.";
-  private IdentityAsserterMessages LOG = MessagesFactory.get(IdentityAsserterMessages.class);
+  private static final IdentityAsserterMessages LOG = MessagesFactory.get(IdentityAsserterMessages.class);
 
+  public static final String VIRTUAL_GROUP_MAPPING_PREFIX = "group.mapping.";
   public static final String GROUP_PRINCIPAL_MAPPING = "group.principal.mapping";
   public static final String PRINCIPAL_MAPPING = "principal.mapping";
-
   private static final String PRINCIPAL_PARAM = "user.name";
   private static final String DOAS_PRINCIPAL_PARAM = "doAs";
+  static final String IMPERSONATION_ENABLED_PARAM = AuthFilterUtils.PROXYUSER_PREFIX + ".impersonation.enabled";
+
   private SimplePrincipalMapper mapper = new SimplePrincipalMapper();
   private final Parser parser = new Parser();
   private VirtualGroupMapper virtualGroupMapper;
   /* List of all default and configured impersonation params */
   protected final List<String> impersonationParamsList = new ArrayList<>();
+  protected boolean impersonationEnabled;
+  private String topologyName;
 
   @Override
   public void init(FilterConfig filterConfig) throws ServletException {
+    topologyName = (String) filterConfig.getServletContext().getAttribute(GatewayServices.GATEWAY_CLUSTER_ATTRIBUTE);
     String principalMapping = filterConfig.getInitParameter(PRINCIPAL_MAPPING);
     if (principalMapping == null || principalMapping.isEmpty()) {
       principalMapping = filterConfig.getServletContext().getInitParameter(PRINCIPAL_MAPPING);
@@ -84,57 +97,83 @@ public class CommonIdentityAssertionFilter extends AbstractIdentityAssertionFilt
         throw new ServletException("Unable to load principal mapping table.", e);
       }
     }
-    virtualGroupMapper = new VirtualGroupMapper(loadVirtualGroups(filterConfig));
-    String impersonationListFromConfig = filterConfig.getInitParameter(IMPERSONATION_PARAMS);
-    if (impersonationListFromConfig == null || impersonationListFromConfig.isEmpty()) {
-      impersonationListFromConfig = filterConfig.getServletContext().getInitParameter(IMPERSONATION_PARAMS);
-    }
-    initImpersonationParamsList(impersonationListFromConfig);
+
+    final List<String> initParameterNames = AuthFilterUtils.getInitParameterNamesAsList(filterConfig);
+
+    virtualGroupMapper = new VirtualGroupMapper(loadVirtualGroups(filterConfig, initParameterNames));
+
+    initImpersonationParamsList(filterConfig);
+    initProxyUserConfiguration(filterConfig, initParameterNames);
   }
 
-  /**
+  /*
    * Initialize the impersonation params list.
    * This list contains query params that needs to be scrubbed
    * from the outgoing request.
-   * @param impersonationListFromConfig
-   * @return
    */
-  private void initImpersonationParamsList(final String impersonationListFromConfig) {
+  private void initImpersonationParamsList(FilterConfig filterConfig) {
+    String impersonationListFromConfig = filterConfig.getInitParameter(IMPERSONATION_PARAMS);
+    if (impersonationListFromConfig == null || impersonationListFromConfig.isEmpty()) {
+      impersonationListFromConfig = filterConfig.getServletContext().getInitParameter(IMPERSONATION_PARAMS);
+    }
+
     /* Add default impersonation params */
     impersonationParamsList.add(DOAS_PRINCIPAL_PARAM);
     impersonationParamsList.add(PRINCIPAL_PARAM);
-    if(null == impersonationListFromConfig || impersonationListFromConfig.isEmpty()) {
-      return;
-    } else {
+
+    if (impersonationListFromConfig != null && !impersonationListFromConfig.isEmpty()) {
       /* Add configured impersonation params */
       LOG.impersonationConfig(impersonationListFromConfig);
       final StringTokenizer t = new StringTokenizer(impersonationListFromConfig, ",");
-      while(t.hasMoreElements()) {
+      while (t.hasMoreElements()) {
         final String token = t.nextToken().trim();
-        if(!impersonationParamsList.contains(token)) {
+        if (!impersonationParamsList.contains(token)) {
           impersonationParamsList.add(token);
         }
       }
     }
   }
 
-  private Map<String, AbstractSyntaxTree> loadVirtualGroups(FilterConfig filterConfig) {
+  private void initProxyUserConfiguration(FilterConfig filterConfig, List<String> initParameterNames) {
+    final String impersonationEnabledValue = filterConfig.getInitParameter(IMPERSONATION_ENABLED_PARAM);
+    impersonationEnabled = impersonationEnabledValue == null ? Boolean.FALSE : Boolean.parseBoolean(impersonationEnabledValue);
+
+    if (impersonationEnabled) {
+      if (AuthFilterUtils.hasProxyConfig(topologyName, "HadoopAuth")) {
+        LOG.ignoreProxyuserConfig();
+        impersonationEnabled = false; //explicitly set to false to avoid redundant authorization attempts at request processing time
+      } else {
+        AuthFilterUtils.refreshSuperUserGroupsConfiguration(filterConfig, initParameterNames, topologyName, ROLE);
+        filterConfig.getServletContext().setAttribute(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE, Boolean.TRUE);
+      }
+    } else {
+      filterConfig.getServletContext().setAttribute(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE, Boolean.FALSE);
+    }
+  }
+
+  boolean isImpersonationEnabled() {
+    return impersonationEnabled;
+  }
+
+  private Map<String, AbstractSyntaxTree> loadVirtualGroups(FilterConfig filterConfig, List<String> initParameterNames) {
     Map<String, AbstractSyntaxTree> predicateToGroupMapping = new HashMap<>();
-    loadVirtualGroupConfig(filterConfig, predicateToGroupMapping);
+    loadVirtualGroupConfig(filterConfig, initParameterNames, predicateToGroupMapping);
     if (predicateToGroupMapping.isEmpty() && filterConfig.getServletContext() != null) {
       loadVirtualGroupConfig(filterConfig.getServletContext(), predicateToGroupMapping);
     }
     return predicateToGroupMapping;
   }
 
-  private void loadVirtualGroupConfig(FilterConfig config, Map<String, AbstractSyntaxTree> result) {
-    for (String paramName : virtualGroupParameterNames(config.getInitParameterNames())) {
+  private void loadVirtualGroupConfig(FilterConfig config, List<String> initParameterNames, Map<String, AbstractSyntaxTree> result) {
+    for (String paramName : virtualGroupParameterNames(initParameterNames)) {
       addGroup(result, paramName, config.getInitParameter(paramName));
     }
   }
 
   private void loadVirtualGroupConfig(ServletContext context, Map<String, AbstractSyntaxTree> result) {
-    for (String paramName : virtualGroupParameterNames(context.getInitParameterNames())) {
+    final List<String> contextInitParams = context.getInitParameterNames() == null ? Collections.emptyList()
+        : Collections.list(context.getInitParameterNames());
+    for (String paramName : virtualGroupParameterNames(contextInitParams)) {
       addGroup(result, paramName, context.getInitParameter(paramName));
     }
   }
@@ -153,18 +192,9 @@ public class CommonIdentityAssertionFilter extends AbstractIdentityAssertionFilt
     }
   }
 
-  private static List<String> virtualGroupParameterNames(Enumeration<String> initParameterNames) {
-    List<String> result = new ArrayList<>();
-    if (initParameterNames == null) {
-      return result;
-    }
-    while (initParameterNames.hasMoreElements()) {
-      String name = initParameterNames.nextElement();
-      if (name.startsWith(VIRTUAL_GROUP_MAPPING_PREFIX)) {
-        result.add(name);
-      }
-    }
-    return result;
+  private static List<String> virtualGroupParameterNames(List<String> initParameterNames) {
+    return initParameterNames == null ? new ArrayList<>()
+        : initParameterNames.stream().filter(name -> name.startsWith(VIRTUAL_GROUP_MAPPING_PREFIX)).collect(Collectors.toList());
   }
 
   @Override
@@ -186,22 +216,47 @@ public class CommonIdentityAssertionFilter extends AbstractIdentityAssertionFilt
       throw new IllegalStateException("Required Subject Missing");
     }
 
-    String principalName = getPrincipalName(subject);
+    String mappedPrincipalName = null;
+    try {
+      mappedPrincipalName = handleProxyUserImpersonation(request, subject);
+    } catch(AuthorizationException e) {
+      LOG.hadoopAuthProxyUserFailed(e);
+      HttpExceptionUtils.createServletExceptionResponse((HttpServletResponse) response, HttpServletResponse.SC_FORBIDDEN, e);
+      return;
+    }
 
-    String mappedPrincipalName = mapUserPrincipalBase(principalName);
+    // mapping principal name using user principal mapping (if configured)
+    mappedPrincipalName = mapUserPrincipalBase(mappedPrincipalName);
     mappedPrincipalName = mapUserPrincipal(mappedPrincipalName);
+
     String[] mappedGroups = mapGroupPrincipalsBase(mappedPrincipalName, subject);
     String[] groups = mapGroupPrincipals(mappedPrincipalName, subject);
     String[] virtualGroups = virtualGroupMapper.mapGroups(mappedPrincipalName, combine(subject, groups), request).toArray(new String[0]);
     groups = combineGroupMappings(mappedGroups, groups);
     groups = combineGroupMappings(virtualGroups, groups);
 
-    HttpServletRequestWrapper wrapper = wrapHttpServletRequest(
-        request, mappedPrincipalName);
+    HttpServletRequestWrapper wrapper = wrapHttpServletRequest(request, mappedPrincipalName);
+
 
     continueChainAsPrincipal(wrapper, response, chain, mappedPrincipalName, unique(groups));
   }
 
+  private String handleProxyUserImpersonation(ServletRequest request, Subject subject) throws AuthorizationException {
+    String principalName = SubjectUtils.getEffectivePrincipalName(subject);
+    if (impersonationEnabled) {
+      final String doAsUser = request.getParameter(AuthFilterUtils.QUERY_PARAMETER_DOAS);
+      if (doAsUser != null && !doAsUser.equals(principalName)) {
+        LOG.hadoopAuthDoAsUser(doAsUser, principalName, request.getRemoteAddr());
+        if (principalName != null) {
+          AuthFilterUtils.authorizeImpersonationRequest((HttpServletRequest) request, principalName, doAsUser, topologyName, ROLE);
+          LOG.hadoopAuthProxyUserSuccess();
+          principalName = doAsUser;
+        }
+      }
+    }
+    return principalName;
+  }
+
   private Set<String> combine(Subject subject, String[] groups) {
     Set<String> result = groups(subject);
     if (groups != null) {
diff --git a/gateway-provider-identity-assertion-common/src/test/java/org/apache/knox/gateway/identityasserter/common/filter/CommonIdentityAssertionFilterTest.java b/gateway-provider-identity-assertion-common/src/test/java/org/apache/knox/gateway/identityasserter/common/filter/CommonIdentityAssertionFilterTest.java
new file mode 100644
index 000000000..f68545be7
--- /dev/null
+++ b/gateway-provider-identity-assertion-common/src/test/java/org/apache/knox/gateway/identityasserter/common/filter/CommonIdentityAssertionFilterTest.java
@@ -0,0 +1,304 @@
+/*
+ * 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.knox.gateway.identityasserter.common.filter;
+
+import static org.apache.knox.gateway.audit.log4j.audit.Log4jAuditService.MDC_AUDIT_CONTEXT_KEY;
+import static org.apache.knox.gateway.identityasserter.common.filter.AbstractIdentityAsserterDeploymentContributor.IMPERSONATION_PARAMS;
+import static org.easymock.EasyMock.createMock;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.nio.charset.StandardCharsets;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Set;
+
+import javax.security.auth.Subject;
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletContext;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletRequestWrapper;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.knox.gateway.context.ContextAttributes;
+import org.apache.knox.gateway.security.GroupPrincipal;
+import org.apache.knox.gateway.security.PrimaryPrincipal;
+import org.apache.knox.gateway.security.SubjectUtils;
+import org.apache.knox.gateway.services.GatewayServices;
+import org.apache.knox.gateway.util.AuthFilterUtils;
+import org.apache.logging.log4j.ThreadContext;
+import org.easymock.EasyMock;
+import org.junit.Before;
+import org.junit.Test;
+
+public class CommonIdentityAssertionFilterTest {
+  private String username;
+  private CommonIdentityAssertionFilter filter;
+  private Set<String> calculatedGroups = new HashSet<>();
+
+  @Before
+  public void setUp() {
+    filter = new CommonIdentityAssertionFilter() {
+      @Override
+      public String mapUserPrincipal(String principalName) {
+        username = principalName.toUpperCase(Locale.ROOT);
+        return principalName;
+      }
+
+      @Override
+      public String[] mapGroupPrincipals(String principalName, Subject subject) {
+        String[] groups = new String[4];
+        int i = 0;
+        for(GroupPrincipal p : subject.getPrincipals(GroupPrincipal.class)) {
+          groups[i] = p.getName().toUpperCase(Locale.ROOT);
+          i++;
+        }
+        return groups;
+      }
+
+      @Override
+      protected String[] combineGroupMappings(String[] mappedGroups, String[] groups) {
+        calculatedGroups.addAll(Arrays.asList(super.combineGroupMappings(mappedGroups, groups)));
+        return super.combineGroupMappings(mappedGroups, groups);
+      }
+
+      @Override
+      protected void continueChainAsPrincipal(HttpServletRequestWrapper request, ServletResponse response, FilterChain chain, String mappedPrincipalName, String[] groups) throws IOException, ServletException {
+        assertEquals("Groups should not have duplicates: " + Arrays.toString(groups),
+                new HashSet<>(Arrays.asList(groups)).size(),
+                groups.length);
+        super.continueChainAsPrincipal(request, response, chain, mappedPrincipalName, groups);
+      }
+    };
+    ThreadContext.put(MDC_AUDIT_CONTEXT_KEY, "dummy");
+  }
+
+  @Test
+  public void testSimpleFilter() throws ServletException, IOException {
+    ServletContext servletContext = createMock(ServletContext.class);
+    EasyMock.expect(servletContext.getAttribute(GatewayServices.GATEWAY_CLUSTER_ATTRIBUTE)).andReturn("topology1").anyTimes();
+    servletContext.setAttribute(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE, Boolean.FALSE);
+    EasyMock.expectLastCall();
+    EasyMock.replay(servletContext);
+    FilterConfig config = EasyMock.createNiceMock( FilterConfig.class );
+    EasyMock.expect(config.getServletContext()).andReturn(servletContext).anyTimes();
+    EasyMock.expect(config.getInitParameter(CommonIdentityAssertionFilter.GROUP_PRINCIPAL_MAPPING)).
+        andReturn("*=everyone;lmccay=test-virtual-group").once();
+    EasyMock.expect(config.getInitParameter(CommonIdentityAssertionFilter.PRINCIPAL_MAPPING)).
+        andReturn("ljm=lmccay;").once();
+    EasyMock.expect(config.getInitParameterNames()).
+            andReturn(Collections.enumeration(Arrays.asList(
+                    CommonIdentityAssertionFilter.GROUP_PRINCIPAL_MAPPING,
+                    CommonIdentityAssertionFilter.PRINCIPAL_MAPPING,
+                    CommonIdentityAssertionFilter.VIRTUAL_GROUP_MAPPING_PREFIX + "test-virtual-group",
+                    CommonIdentityAssertionFilter.VIRTUAL_GROUP_MAPPING_PREFIX))) // invalid group with no name
+            .anyTimes();
+    EasyMock.expect(config.getInitParameter(IMPERSONATION_PARAMS)).
+        andReturn("doAs").anyTimes();
+    EasyMock.expect(config.getInitParameter(CommonIdentityAssertionFilter.VIRTUAL_GROUP_MAPPING_PREFIX + "test-virtual-group")).
+            andReturn("(and (username 'lmccay') (and (member 'users') (member 'admin')))").anyTimes();
+    EasyMock.expect(config.getInitParameter(CommonIdentityAssertionFilter.VIRTUAL_GROUP_MAPPING_PREFIX)).
+            andReturn("true").anyTimes();
+    EasyMock.replay( config );
+
+    final HttpServletRequest request = EasyMock.createNiceMock( HttpServletRequest.class );
+    EasyMock.replay( request );
+
+    final HttpServletResponse response = EasyMock.createNiceMock( HttpServletResponse.class );
+    EasyMock.replay( response );
+
+    final FilterChain chain = (req, resp) -> {};
+
+    Subject subject = new Subject();
+    subject.getPrincipals().add(new PrimaryPrincipal("ljm"));
+    subject.getPrincipals().add(new GroupPrincipal("users"));
+    subject.getPrincipals().add(new GroupPrincipal("admin"));
+    try {
+      Subject.doAs(
+        subject,
+              (PrivilegedExceptionAction<Object>) () -> {
+                filter.init(config);
+                filter.doFilter(request, response, chain);
+                return null;
+              });
+    }
+    catch (PrivilegedActionException e) {
+      Throwable t = e.getCause();
+      if (t instanceof IOException) {
+        throw (IOException) t;
+      }
+      else if (t instanceof ServletException) {
+        throw (ServletException) t;
+      }
+      else {
+        throw new ServletException(t);
+      }
+    }
+
+    assertEquals("LMCCAY", username);
+    assertTrue("Should be greater than 2", calculatedGroups.size() > 2);
+    assertTrue(calculatedGroups.containsAll(Arrays.asList("everyone", "USERS", "ADMIN", "test-virtual-group")));
+    assertFalse(calculatedGroups.contains(""));
+  }
+
+  @Test
+  public void testProxyUserImpersonationDisabled() throws Exception {
+    testProxyUserImpersonationEnabled(false);
+
+  }
+
+  @Test
+  public void testProxyUserImpersonationEnabled() throws Exception {
+    testProxyUserImpersonationEnabled(true);
+  }
+
+  @Test
+  public void testProxyUserImpersonationEnabledAndConfiguredInHadoopAuth() throws Exception {
+    testProxyUserImpersonationEnabled(true, true, Arrays.asList());
+  }
+
+  @Test
+  public void testProxyUserImpersonationFailWithoutProxyUserConfig() throws Exception {
+    final String impersonatedUser = "bob";
+
+    // enable impersonation without configuring context/filter expectations
+    testProxyUserImpersonationEnabled(true);
+
+    final HttpServletRequest request = EasyMock.createNiceMock(HttpServletRequest.class);
+    EasyMock.expect(request.getParameter(AuthFilterUtils.QUERY_PARAMETER_DOAS)).andReturn(impersonatedUser).once();
+
+    final ByteArrayOutputStream out = new ByteArrayOutputStream();
+    final PrintWriter writer = new PrintWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8));
+    final HttpServletResponse response = EasyMock.createNiceMock(HttpServletResponse.class);
+    EasyMock.expect(response.getWriter()).andReturn(writer).anyTimes();
+    EasyMock.replay(request, response);
+
+    final FilterChainWrapper chain = new FilterChainWrapper((req, resp) -> {
+    });
+
+    final Subject subject = new Subject();
+    subject.getPrincipals().add(new PrimaryPrincipal("admin"));
+    Subject.doAs(subject, (PrivilegedExceptionAction<Object>) () -> {
+      filter.doFilter(request, response, chain);
+      return null;
+    });
+    assertTrue(new String(out.toByteArray(), StandardCharsets.UTF_8).contains("User: admin is not allowed to impersonate bob"));
+  }
+
+  @Test
+  public void testProxyUserImpersonationApplied() throws Exception {
+    final String impersonatedUser = "bob";
+
+    // enable impersonation and configure context/filter expectations
+    final List<String> proxyUserConfig = Arrays.asList(AuthFilterUtils.PROXYUSER_PREFIX + ".admin.users", AuthFilterUtils.PROXYUSER_PREFIX + ".admin.hosts");
+    testProxyUserImpersonationEnabled(true, false, proxyUserConfig);
+
+    final HttpServletRequest request = EasyMock.createNiceMock(HttpServletRequest.class);
+    EasyMock.expect(request.getParameter(AuthFilterUtils.QUERY_PARAMETER_DOAS)).andReturn(impersonatedUser).once();
+    final HttpServletResponse response = EasyMock.createNiceMock(HttpServletResponse.class);
+    EasyMock.replay(request, response);
+
+    final FilterChainWrapper chain = new FilterChainWrapper((req, resp) -> {
+    });
+
+    final Subject subject = new Subject();
+    subject.getPrincipals().add(new PrimaryPrincipal("admin"));
+    Subject.doAs(subject, (PrivilegedExceptionAction<Object>) () -> {
+      filter.doFilter(request, response, chain);
+      return null;
+    });
+    assertTrue(filter.isImpersonationEnabled());
+    final Subject impersonatedSubject = chain.getSubject();
+    assertTrue(SubjectUtils.isImpersonating(impersonatedSubject));
+    assertEquals("admin", SubjectUtils.getPrimaryPrincipalName(impersonatedSubject));
+    assertEquals("bob", SubjectUtils.getEffectivePrincipalName(impersonatedSubject));
+  }
+
+  private static final class FilterChainWrapper implements FilterChain {
+    private final FilterChain wrappedFilterChain;
+    private Subject subject;
+
+    FilterChainWrapper(FilterChain wrappedFilterChain) {
+      this.wrappedFilterChain = wrappedFilterChain;
+    }
+
+    @Override
+    public void doFilter(ServletRequest request, ServletResponse response) throws IOException, ServletException {
+      this.subject = SubjectUtils.getCurrentSubject();
+      wrappedFilterChain.doFilter(request, response);
+    }
+
+    Subject getSubject() {
+      return subject;
+    }
+  }
+
+  private void testProxyUserImpersonationEnabled(boolean impersonationEnabled) throws Exception {
+    testProxyUserImpersonationEnabled(impersonationEnabled, false, Arrays.asList());
+  }
+
+  private void testProxyUserImpersonationEnabled(boolean impersonationEnabled, boolean configuredInHadoopAuth, List<String> filterConfigParameterNames)
+      throws Exception {
+    ServletContext servletContext = createMock(ServletContext.class);
+    EasyMock.expect(servletContext.getAttribute(GatewayServices.GATEWAY_CLUSTER_ATTRIBUTE)).andReturn("topology1").anyTimes();
+    EasyMock.expect(servletContext.getInitParameter(CommonIdentityAssertionFilter.PRINCIPAL_MAPPING)).andReturn(null).anyTimes();
+    EasyMock.expect(servletContext.getInitParameter(CommonIdentityAssertionFilter.GROUP_PRINCIPAL_MAPPING)).andReturn(null).anyTimes();
+    EasyMock.expect(servletContext.getInitParameterNames()).andReturn(Collections.enumeration(filterConfigParameterNames)).anyTimes();
+    EasyMock.expect(servletContext.getInitParameter(IMPERSONATION_PARAMS)).andReturn("doAs").anyTimes();
+    if (!configuredInHadoopAuth) {
+      servletContext.setAttribute(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE, Boolean.valueOf(impersonationEnabled));
+      EasyMock.expectLastCall();
+    }
+    final FilterConfig filterConfig = EasyMock.createNiceMock(FilterConfig.class);
+    final String impersonatedEnabledParam = impersonationEnabled ? Boolean.TRUE.toString() : Boolean.FALSE.toString();
+    EasyMock.expect(filterConfig.getInitParameter(CommonIdentityAssertionFilter.IMPERSONATION_ENABLED_PARAM)).andReturn(impersonatedEnabledParam);
+    EasyMock.expect(filterConfig.getInitParameter(CommonIdentityAssertionFilter.PRINCIPAL_MAPPING)).andReturn(null).anyTimes();
+    EasyMock.expect(filterConfig.getInitParameter(CommonIdentityAssertionFilter.GROUP_PRINCIPAL_MAPPING)).andReturn(null).anyTimes();
+    EasyMock.expect(filterConfig.getInitParameterNames()).andReturn(Collections.enumeration(filterConfigParameterNames)).atLeastOnce();
+    filterConfigParameterNames.forEach(filterConfigParameterName -> {
+      EasyMock.expect(filterConfig.getInitParameter(filterConfigParameterName)).andReturn("*").anyTimes();
+    });
+    EasyMock.expect(filterConfig.getServletContext()).andReturn(servletContext).anyTimes();
+    EasyMock.replay(servletContext, filterConfig);
+
+    if (configuredInHadoopAuth) {
+      AuthFilterUtils.refreshSuperUserGroupsConfiguration(filterConfig, Arrays.asList(), "topology1", "HadoopAuth");
+    } else {
+      AuthFilterUtils.removeProxyUserConfig("topology1", "HadoopAuth");
+    }
+
+    filter.init(filterConfig);
+    assertEquals(impersonationEnabled && !configuredInHadoopAuth, filter.isImpersonationEnabled());
+    EasyMock.verify(servletContext, filterConfig);
+  }
+
+}
diff --git a/gateway-provider-identity-assertion-common/src/test/java/org/apache/knox/gateway/identityasserter/filter/CommonIdentityAssertionFilterTest.java b/gateway-provider-identity-assertion-common/src/test/java/org/apache/knox/gateway/identityasserter/filter/CommonIdentityAssertionFilterTest.java
deleted file mode 100644
index 5d5d7806d..000000000
--- a/gateway-provider-identity-assertion-common/src/test/java/org/apache/knox/gateway/identityasserter/filter/CommonIdentityAssertionFilterTest.java
+++ /dev/null
@@ -1,155 +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.knox.gateway.identityasserter.filter;
-
-import static org.apache.knox.gateway.audit.log4j.audit.Log4jAuditService.MDC_AUDIT_CONTEXT_KEY;
-import static org.apache.knox.gateway.identityasserter.common.filter.AbstractIdentityAsserterDeploymentContributor.IMPERSONATION_PARAMS;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.security.PrivilegedActionException;
-import java.security.PrivilegedExceptionAction;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Locale;
-import java.util.Set;
-import javax.security.auth.Subject;
-import javax.servlet.Filter;
-import javax.servlet.FilterChain;
-import javax.servlet.FilterConfig;
-import javax.servlet.ServletException;
-import javax.servlet.ServletResponse;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletRequestWrapper;
-import javax.servlet.http.HttpServletResponse;
-
-import org.apache.knox.gateway.identityasserter.common.filter.CommonIdentityAssertionFilter;
-import org.apache.knox.gateway.security.GroupPrincipal;
-import org.apache.knox.gateway.security.PrimaryPrincipal;
-import org.apache.logging.log4j.ThreadContext;
-import org.easymock.EasyMock;
-import org.junit.Before;
-import org.junit.Test;
-
-public class CommonIdentityAssertionFilterTest {
-  private String username;
-  private Filter filter;
-  private Set<String> calculatedGroups = new HashSet<>();
-
-  @Before
-  public void setUp() {
-    filter = new CommonIdentityAssertionFilter() {
-      @Override
-      public String mapUserPrincipal(String principalName) {
-        username = principalName.toUpperCase(Locale.ROOT);
-        return principalName;
-      }
-
-      @Override
-      public String[] mapGroupPrincipals(String principalName, Subject subject) {
-        String[] groups = new String[4];
-        int i = 0;
-        for(GroupPrincipal p : subject.getPrincipals(GroupPrincipal.class)) {
-          groups[i] = p.getName().toUpperCase(Locale.ROOT);
-          i++;
-        }
-        return groups;
-      }
-
-      @Override
-      protected String[] combineGroupMappings(String[] mappedGroups, String[] groups) {
-        calculatedGroups.addAll(Arrays.asList(super.combineGroupMappings(mappedGroups, groups)));
-        return super.combineGroupMappings(mappedGroups, groups);
-      }
-
-      @Override
-      protected void continueChainAsPrincipal(HttpServletRequestWrapper request, ServletResponse response, FilterChain chain, String mappedPrincipalName, String[] groups) throws IOException, ServletException {
-        assertEquals("Groups should not have duplicates: " + Arrays.toString(groups),
-                new HashSet<>(Arrays.asList(groups)).size(),
-                groups.length);
-        super.continueChainAsPrincipal(request, response, chain, mappedPrincipalName, groups);
-      }
-    };
-    ThreadContext.put(MDC_AUDIT_CONTEXT_KEY, "dummy");
-  }
-
-  @Test
-  public void testSimpleFilter() throws ServletException, IOException {
-    FilterConfig config = EasyMock.createNiceMock( FilterConfig.class );
-    EasyMock.expect(config.getInitParameter(CommonIdentityAssertionFilter.GROUP_PRINCIPAL_MAPPING)).
-        andReturn("*=everyone;lmccay=test-virtual-group").once();
-    EasyMock.expect(config.getInitParameter(CommonIdentityAssertionFilter.PRINCIPAL_MAPPING)).
-        andReturn("ljm=lmccay;").once();
-    EasyMock.expect(config.getInitParameterNames()).
-            andReturn(Collections.enumeration(Arrays.asList(
-                    CommonIdentityAssertionFilter.GROUP_PRINCIPAL_MAPPING,
-                    CommonIdentityAssertionFilter.PRINCIPAL_MAPPING,
-                    CommonIdentityAssertionFilter.VIRTUAL_GROUP_MAPPING_PREFIX + "test-virtual-group",
-                    CommonIdentityAssertionFilter.VIRTUAL_GROUP_MAPPING_PREFIX))) // invalid group with no name
-            .anyTimes();
-    EasyMock.expect(config.getInitParameter(IMPERSONATION_PARAMS)).
-        andReturn("doAs").anyTimes();
-    EasyMock.expect(config.getInitParameter(CommonIdentityAssertionFilter.VIRTUAL_GROUP_MAPPING_PREFIX + "test-virtual-group")).
-            andReturn("(and (username 'lmccay') (and (member 'users') (member 'admin')))").anyTimes();
-    EasyMock.expect(config.getInitParameter(CommonIdentityAssertionFilter.VIRTUAL_GROUP_MAPPING_PREFIX)).
-            andReturn("true").anyTimes();
-    EasyMock.replay( config );
-
-    final HttpServletRequest request = EasyMock.createNiceMock( HttpServletRequest.class );
-    EasyMock.replay( request );
-
-    final HttpServletResponse response = EasyMock.createNiceMock( HttpServletResponse.class );
-    EasyMock.replay( response );
-
-    final FilterChain chain = (req, resp) -> {};
-
-    Subject subject = new Subject();
-    subject.getPrincipals().add(new PrimaryPrincipal("ljm"));
-    subject.getPrincipals().add(new GroupPrincipal("users"));
-    subject.getPrincipals().add(new GroupPrincipal("admin"));
-    try {
-      Subject.doAs(
-        subject,
-              (PrivilegedExceptionAction<Object>) () -> {
-                filter.init(config);
-                filter.doFilter(request, response, chain);
-                return null;
-              });
-    }
-    catch (PrivilegedActionException e) {
-      Throwable t = e.getCause();
-      if (t instanceof IOException) {
-        throw (IOException) t;
-      }
-      else if (t instanceof ServletException) {
-        throw (ServletException) t;
-      }
-      else {
-        throw new ServletException(t);
-      }
-    }
-
-    assertEquals("LMCCAY", username);
-    assertTrue("Should be greater than 2", calculatedGroups.size() > 2);
-    assertTrue(calculatedGroups.containsAll(Arrays.asList("everyone", "USERS", "ADMIN", "test-virtual-group")));
-    assertFalse(calculatedGroups.contains(""));
-  }
-}
diff --git a/gateway-provider-identity-assertion-concat/src/test/java/org/apache/knox/gateway/identityasserter/concat/filter/ConcatIdentityAssertionFilterTest.java b/gateway-provider-identity-assertion-concat/src/test/java/org/apache/knox/gateway/identityasserter/concat/filter/ConcatIdentityAssertionFilterTest.java
index 4fbe020fb..6286158fc 100644
--- a/gateway-provider-identity-assertion-concat/src/test/java/org/apache/knox/gateway/identityasserter/concat/filter/ConcatIdentityAssertionFilterTest.java
+++ b/gateway-provider-identity-assertion-concat/src/test/java/org/apache/knox/gateway/identityasserter/concat/filter/ConcatIdentityAssertionFilterTest.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 
 import java.security.Principal;
+import java.util.Collections;
 
 import javax.security.auth.Subject;
 import javax.servlet.FilterConfig;
@@ -38,7 +39,9 @@ public class ConcatIdentityAssertionFilterTest {
     FilterConfig config = EasyMock.createNiceMock( FilterConfig.class );
     EasyMock.expect(config.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     ServletContext context = EasyMock.createNiceMock(ServletContext.class);
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     EasyMock.replay( config );
     EasyMock.replay( context );
@@ -59,7 +62,9 @@ public class ConcatIdentityAssertionFilterTest {
     config = EasyMock.createNiceMock( FilterConfig.class );
     EasyMock.expect(config.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     context = EasyMock.createNiceMock(ServletContext.class);
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     EasyMock.expect(config.getInitParameter("concat.prefix") ).andReturn( "sir-" ).anyTimes();
     EasyMock.replay( config );
@@ -70,7 +75,9 @@ public class ConcatIdentityAssertionFilterTest {
 
     config = EasyMock.createNiceMock( FilterConfig.class );
     context = EasyMock.createNiceMock(ServletContext.class);
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     EasyMock.expect(config.getInitParameter("concat.suffix") ).andReturn( "-tenant-1" ).anyTimes();
     EasyMock.replay( config );
@@ -81,7 +88,9 @@ public class ConcatIdentityAssertionFilterTest {
 
     config = EasyMock.createNiceMock( FilterConfig.class );
     context = EasyMock.createNiceMock(ServletContext.class);
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     EasyMock.expect(config.getInitParameter("concat.prefix") ).andReturn( "sir-" ).anyTimes();
     EasyMock.expect(config.getInitParameter("concat.suffix") ).andReturn( "-tenant-1" ).anyTimes();
diff --git a/gateway-provider-identity-assertion-hadoop-groups/src/test/java/org/apache/knox/gateway/identityasserter/hadoop/groups/filter/HadoopGroupProviderFilterTest.java b/gateway-provider-identity-assertion-hadoop-groups/src/test/java/org/apache/knox/gateway/identityasserter/hadoop/groups/filter/HadoopGroupProviderFilterTest.java
index 5ed4bc97c..8e49ad1bb 100644
--- a/gateway-provider-identity-assertion-hadoop-groups/src/test/java/org/apache/knox/gateway/identityasserter/hadoop/groups/filter/HadoopGroupProviderFilterTest.java
+++ b/gateway-provider-identity-assertion-hadoop-groups/src/test/java/org/apache/knox/gateway/identityasserter/hadoop/groups/filter/HadoopGroupProviderFilterTest.java
@@ -77,7 +77,9 @@ public class HadoopGroupProviderFilterTest {
     final FilterConfig config = EasyMock.createNiceMock(FilterConfig.class);
     EasyMock.expect(config.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     ServletContext context = EasyMock.createNiceMock(ServletContext.class);
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     EasyMock.replay( config );
     EasyMock.replay( context );
@@ -109,7 +111,9 @@ public class HadoopGroupProviderFilterTest {
     final FilterConfig config = EasyMock.createNiceMock(FilterConfig.class);
     EasyMock.expect(config.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     ServletContext context = EasyMock.createNiceMock(ServletContext.class);
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     EasyMock.replay( config );
     EasyMock.replay( context );
@@ -153,6 +157,7 @@ public class HadoopGroupProviderFilterTest {
     final FilterConfig config = EasyMock.createNiceMock(FilterConfig.class);
     EasyMock.expect(config.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     ServletContext context = EasyMock.createNiceMock(ServletContext.class);
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
 
@@ -216,6 +221,7 @@ public class HadoopGroupProviderFilterTest {
     Set<String> calculatedGroups = new HashSet<>();
     FilterConfig config = EasyMock.createNiceMock(FilterConfig.class);
     ServletContext context = EasyMock.createNiceMock(ServletContext.class);
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(config.getServletContext()).andReturn(context).anyTimes();
     EasyMock.expect(config.getInitParameterNames()).
             andReturn(Collections.enumeration(Arrays.asList(
diff --git a/gateway-provider-identity-assertion-no-doas/src/test/java/org/apache/knox/gateway/identityasserter/filter/NoImpersonationFilterTest.java b/gateway-provider-identity-assertion-no-doas/src/test/java/org/apache/knox/gateway/identityasserter/filter/NoImpersonationFilterTest.java
index 8566b7524..197af20d3 100644
--- a/gateway-provider-identity-assertion-no-doas/src/test/java/org/apache/knox/gateway/identityasserter/filter/NoImpersonationFilterTest.java
+++ b/gateway-provider-identity-assertion-no-doas/src/test/java/org/apache/knox/gateway/identityasserter/filter/NoImpersonationFilterTest.java
@@ -17,8 +17,10 @@
  */
 package org.apache.knox.gateway.identityasserter.filter;
 
+import org.apache.knox.gateway.context.ContextAttributes;
 import org.apache.knox.gateway.security.GroupPrincipal;
 import org.apache.knox.gateway.security.PrimaryPrincipal;
+import org.apache.knox.gateway.services.GatewayServices;
 import org.easymock.EasyMock;
 import org.junit.Test;
 
@@ -27,6 +29,7 @@ import javax.servlet.FilterConfig;
 import javax.servlet.ServletContext;
 import java.security.Principal;
 import java.util.Arrays;
+import java.util.Collections;
 
 import static org.apache.knox.gateway.identityasserter.common.filter.AbstractIdentityAsserterDeploymentContributor.IMPERSONATION_PARAMS;
 import static org.junit.Assert.assertEquals;
@@ -41,7 +44,12 @@ public class NoImpersonationFilterTest {
     FilterConfig config = EasyMock.createNiceMock( FilterConfig.class );
     EasyMock.expect(config.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     ServletContext context = EasyMock.createNiceMock(ServletContext.class);
+    EasyMock.expect(context.getAttribute(GatewayServices.GATEWAY_CLUSTER_ATTRIBUTE)).andReturn("topology1").anyTimes();
+    context.setAttribute(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE, Boolean.FALSE);
+    EasyMock.expectLastCall();
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     EasyMock.replay( config );
     EasyMock.replay( context );
@@ -64,8 +72,13 @@ public class NoImpersonationFilterTest {
     EasyMock.expect(config.getInitParameter("group.principal.mapping") ).andReturn( "kminder=group1;lmccay=mrgroup,mrducks" ).anyTimes();
     EasyMock.expect(config.getInitParameter(IMPERSONATION_PARAMS) ).andReturn("doAs").anyTimes();
     context = EasyMock.createNiceMock(ServletContext.class);
+    EasyMock.expect(context.getAttribute(GatewayServices.GATEWAY_CLUSTER_ATTRIBUTE)).andReturn("topology1").anyTimes();
+    context.setAttribute(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE, Boolean.FALSE);
+    EasyMock.expectLastCall();
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
-    EasyMock.replay( config );
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.replay( context, config );
     filter.init(config);
     username = filter.mapUserPrincipal(((Principal) subject.getPrincipals(PrimaryPrincipal.class).toArray()[0]).getName());
     String[] mappedGroups = filter.mapGroupPrincipals(((Principal) subject.getPrincipals(PrimaryPrincipal.class).toArray()[0]).getName(), subject);
@@ -85,8 +98,13 @@ public class NoImpersonationFilterTest {
     EasyMock.expect(config.getInitParameter("group.principal.mapping") ).andReturn( "kminder=group1;lmccay=mrgroup,mrducks" ).anyTimes();
     EasyMock.expect(config.getInitParameter(IMPERSONATION_PARAMS) ).andReturn("doAs").anyTimes();
     context = EasyMock.createNiceMock(ServletContext.class);
+    EasyMock.expect(context.getAttribute(GatewayServices.GATEWAY_CLUSTER_ATTRIBUTE)).andReturn("topology1").anyTimes();
+    context.setAttribute(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE, Boolean.FALSE);
+    EasyMock.expectLastCall();
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
-    EasyMock.replay( config );
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.replay(context, config );
     filter.init(config);
     username = filter.mapUserPrincipal(((Principal) subject.getPrincipals(PrimaryPrincipal.class).toArray()[0]).getName());
     mappedGroups = filter.mapGroupPrincipals(((Principal) subject.getPrincipals(PrimaryPrincipal.class).toArray()[0]).getName(), subject);
@@ -116,6 +134,8 @@ public class NoImpersonationFilterTest {
     EasyMock.expect(config.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     ServletContext context = EasyMock.createNiceMock(ServletContext.class);
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.replay( config );
     EasyMock.replay( context );
 
@@ -137,6 +157,8 @@ public class NoImpersonationFilterTest {
     EasyMock.expect(config.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     context = EasyMock.createNiceMock(ServletContext.class);
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "lmccay,kminder=hdfs;newuser=mapred" ).anyTimes();
     EasyMock.expect(context.getInitParameter("group.principal.mapping") ).andReturn( "kminder=group1;lmccay=mrgroup,mrducks" ).anyTimes();
     EasyMock.replay( config );
@@ -159,6 +181,8 @@ public class NoImpersonationFilterTest {
     EasyMock.expect(config.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     context = EasyMock.createNiceMock(ServletContext.class);
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "lmccay,kminder=hdfs;newuser=mapred" ).anyTimes();
     EasyMock.expect(context.getInitParameter("group.principal.mapping") ).andReturn( "kminder=group1;lmccay=mrgroup,mrducks" ).anyTimes();
     EasyMock.replay( config );
diff --git a/gateway-provider-identity-assertion-pseudo/src/test/java/org/apache/knox/gateway/identityasserter/filter/DefaultIdentityAssertionFilterTest.java b/gateway-provider-identity-assertion-pseudo/src/test/java/org/apache/knox/gateway/identityasserter/filter/DefaultIdentityAssertionFilterTest.java
index 1b26bfb44..6ba32b298 100644
--- a/gateway-provider-identity-assertion-pseudo/src/test/java/org/apache/knox/gateway/identityasserter/filter/DefaultIdentityAssertionFilterTest.java
+++ b/gateway-provider-identity-assertion-pseudo/src/test/java/org/apache/knox/gateway/identityasserter/filter/DefaultIdentityAssertionFilterTest.java
@@ -17,8 +17,10 @@
  */
 package org.apache.knox.gateway.identityasserter.filter;
 
+import org.apache.knox.gateway.context.ContextAttributes;
 import org.apache.knox.gateway.security.GroupPrincipal;
 import org.apache.knox.gateway.security.PrimaryPrincipal;
+import org.apache.knox.gateway.services.GatewayServices;
 import org.easymock.EasyMock;
 import org.junit.Test;
 
@@ -27,6 +29,7 @@ import javax.servlet.FilterConfig;
 import javax.servlet.ServletContext;
 import java.security.Principal;
 import java.util.Arrays;
+import java.util.Collections;
 
 import static org.apache.knox.gateway.identityasserter.common.filter.AbstractIdentityAsserterDeploymentContributor.IMPERSONATION_PARAMS;
 import static org.junit.Assert.assertEquals;
@@ -42,7 +45,12 @@ public class DefaultIdentityAssertionFilterTest {
     EasyMock.expect(config.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     ServletContext context = EasyMock.createNiceMock(ServletContext.class);
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
+    EasyMock.expect(context.getAttribute(GatewayServices.GATEWAY_CLUSTER_ATTRIBUTE)).andReturn("topology1").anyTimes();
+    context.setAttribute(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE, Boolean.FALSE);
+    EasyMock.expectLastCall();
     EasyMock.replay( config );
     EasyMock.replay( context );
 
@@ -64,8 +72,13 @@ public class DefaultIdentityAssertionFilterTest {
     EasyMock.expect(config.getInitParameter("group.principal.mapping") ).andReturn( "kminder=group1;lmccay=mrgroup,mrducks" ).anyTimes();
     EasyMock.expect(config.getInitParameter(IMPERSONATION_PARAMS) ).andReturn("doAs").anyTimes();
     context = EasyMock.createNiceMock(ServletContext.class);
+    EasyMock.expect(context.getAttribute(GatewayServices.GATEWAY_CLUSTER_ATTRIBUTE)).andReturn("topology1").anyTimes();
+    context.setAttribute(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE, Boolean.FALSE);
+    EasyMock.expectLastCall();
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
-    EasyMock.replay( config );
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.replay( context, config );
     filter.init(config);
     username = filter.mapUserPrincipal(((Principal) subject.getPrincipals(PrimaryPrincipal.class).toArray()[0]).getName());
     String[] mappedGroups = filter.mapGroupPrincipals(((Principal) subject.getPrincipals(PrimaryPrincipal.class).toArray()[0]).getName(), subject);
@@ -85,8 +98,13 @@ public class DefaultIdentityAssertionFilterTest {
     EasyMock.expect(config.getInitParameter("group.principal.mapping") ).andReturn( "kminder=group1;lmccay=mrgroup,mrducks" ).anyTimes();
     EasyMock.expect(config.getInitParameter(IMPERSONATION_PARAMS) ).andReturn("doAs").anyTimes();
     context = EasyMock.createNiceMock(ServletContext.class);
+    EasyMock.expect(context.getAttribute(GatewayServices.GATEWAY_CLUSTER_ATTRIBUTE)).andReturn("topology1").anyTimes();
+    context.setAttribute(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE, Boolean.FALSE);
+    EasyMock.expectLastCall();
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
-    EasyMock.replay( config );
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.replay( context, config );
     filter.init(config);
     username = filter.mapUserPrincipal(((Principal) subject.getPrincipals(PrimaryPrincipal.class).toArray()[0]).getName());
     mappedGroups = filter.mapGroupPrincipals(((Principal) subject.getPrincipals(PrimaryPrincipal.class).toArray()[0]).getName(), subject);
@@ -116,6 +134,8 @@ public class DefaultIdentityAssertionFilterTest {
     EasyMock.expect(config.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     ServletContext context = EasyMock.createNiceMock(ServletContext.class);
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.replay( config );
     EasyMock.replay( context );
 
@@ -137,6 +157,8 @@ public class DefaultIdentityAssertionFilterTest {
     EasyMock.expect(config.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     context = EasyMock.createNiceMock(ServletContext.class);
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "lmccay,kminder=hdfs;newuser=mapred" ).anyTimes();
     EasyMock.expect(context.getInitParameter("group.principal.mapping") ).andReturn( "kminder=group1;lmccay=mrgroup,mrducks" ).anyTimes();
     EasyMock.replay( config );
@@ -159,6 +181,8 @@ public class DefaultIdentityAssertionFilterTest {
     EasyMock.expect(config.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     context = EasyMock.createNiceMock(ServletContext.class);
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "lmccay,kminder=hdfs;newuser=mapred" ).anyTimes();
     EasyMock.expect(context.getInitParameter("group.principal.mapping") ).andReturn( "kminder=group1;lmccay=mrgroup,mrducks" ).anyTimes();
     EasyMock.replay( config );
diff --git a/gateway-provider-identity-assertion-regex/src/test/java/org/apache/knox/gateway/identityasserter/regex/filter/RegexIdentityAssertionFilterTest.java b/gateway-provider-identity-assertion-regex/src/test/java/org/apache/knox/gateway/identityasserter/regex/filter/RegexIdentityAssertionFilterTest.java
index 859bf852d..21d77cc72 100644
--- a/gateway-provider-identity-assertion-regex/src/test/java/org/apache/knox/gateway/identityasserter/regex/filter/RegexIdentityAssertionFilterTest.java
+++ b/gateway-provider-identity-assertion-regex/src/test/java/org/apache/knox/gateway/identityasserter/regex/filter/RegexIdentityAssertionFilterTest.java
@@ -27,6 +27,7 @@ import javax.servlet.FilterConfig;
 import javax.servlet.ServletContext;
 
 import java.security.Principal;
+import java.util.Collections;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.CoreMatchers.nullValue;
@@ -40,6 +41,8 @@ public class RegexIdentityAssertionFilterTest {
     FilterConfig config = EasyMock.createNiceMock( FilterConfig.class );
     EasyMock.expect(config.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     ServletContext context = EasyMock.createNiceMock(ServletContext.class);
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     EasyMock.replay( config );
@@ -64,6 +67,8 @@ public class RegexIdentityAssertionFilterTest {
     EasyMock.expect(config.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     context = EasyMock.createNiceMock(ServletContext.class);
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     EasyMock.expect(config.getInitParameter( "output" ) ).andReturn( "test-output" ).anyTimes();
     EasyMock.replay( config );
@@ -77,6 +82,8 @@ public class RegexIdentityAssertionFilterTest {
     EasyMock.expect(config.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     context = EasyMock.createNiceMock(ServletContext.class);
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     EasyMock.expect(config.getInitParameter( "input" ) ).andReturn( "(.*)@.*" ).anyTimes();
     EasyMock.expect(config.getInitParameter( "output" ) ).andReturn( "prefix_{1}_suffix" ).anyTimes();
@@ -94,6 +101,8 @@ public class RegexIdentityAssertionFilterTest {
     EasyMock.expect(config.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     ServletContext context = EasyMock.createNiceMock(ServletContext.class);
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     EasyMock.replay( config );
     EasyMock.replay( context );
@@ -112,6 +121,8 @@ public class RegexIdentityAssertionFilterTest {
     EasyMock.expect(config.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     context = EasyMock.createNiceMock(ServletContext.class);
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     EasyMock.expect(config.getInitParameter( "input" ) ).andReturn( "(.*)@(.*?)\\..*" ).anyTimes();
     EasyMock.expect(config.getInitParameter( "output" ) ).andReturn( "prefix_{1}_suffix:{[2]}" ).anyTimes();
@@ -139,6 +150,8 @@ public class RegexIdentityAssertionFilterTest {
     EasyMock.expect(config.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     context = EasyMock.createNiceMock(ServletContext.class);
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     EasyMock.expect(config.getInitParameter( "input" ) ).andReturn( "([^@]*)(@.*)?" ).anyTimes();
     EasyMock.expect(config.getInitParameter( "output" ) ).andReturn( "prefix_{1}_suffix" ).anyTimes();
diff --git a/gateway-provider-identity-assertion-switchcase/src/test/java/org/apache/knox/gateway/identityasserter/switchcase/SwitchCaseIdentityAssertionFilterTest.java b/gateway-provider-identity-assertion-switchcase/src/test/java/org/apache/knox/gateway/identityasserter/switchcase/SwitchCaseIdentityAssertionFilterTest.java
index 87ad85257..bfb25ea57 100644
--- a/gateway-provider-identity-assertion-switchcase/src/test/java/org/apache/knox/gateway/identityasserter/switchcase/SwitchCaseIdentityAssertionFilterTest.java
+++ b/gateway-provider-identity-assertion-switchcase/src/test/java/org/apache/knox/gateway/identityasserter/switchcase/SwitchCaseIdentityAssertionFilterTest.java
@@ -18,6 +18,8 @@
 package org.apache.knox.gateway.identityasserter.switchcase;
 
 import java.security.Principal;
+import java.util.Collections;
+
 import javax.security.auth.Subject;
 import javax.servlet.FilterConfig;
 import javax.servlet.ServletContext;
@@ -40,6 +42,8 @@ public class SwitchCaseIdentityAssertionFilterTest {
     EasyMock.expect(config.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     ServletContext context = EasyMock.createNiceMock(ServletContext.class);
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     EasyMock.replay( config );
     EasyMock.replay( context );
@@ -67,6 +71,8 @@ public class SwitchCaseIdentityAssertionFilterTest {
     EasyMock.expect(config.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     ServletContext context = EasyMock.createNiceMock(ServletContext.class);
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     EasyMock.replay( config );
     EasyMock.replay( context );
@@ -93,6 +99,8 @@ public class SwitchCaseIdentityAssertionFilterTest {
     EasyMock.expect( config.getInitParameter( "group.principal.case" ) ).andReturn( "LOWER" ).anyTimes();
     ServletContext context = EasyMock.createNiceMock(ServletContext.class);
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     EasyMock.replay( config );
     EasyMock.replay( context );
@@ -119,6 +127,8 @@ public class SwitchCaseIdentityAssertionFilterTest {
     EasyMock.expect( config.getInitParameter( "group.principal.case" ) ).andReturn( "NONE" ).anyTimes();
     ServletContext context = EasyMock.createNiceMock(ServletContext.class);
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     EasyMock.replay( config );
     EasyMock.replay( context );
@@ -143,6 +153,8 @@ public class SwitchCaseIdentityAssertionFilterTest {
     EasyMock.expect( config.getInitParameter( "group.principal.case" ) ).andReturn( null ).anyTimes();
     ServletContext context = EasyMock.createNiceMock(ServletContext.class);
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     EasyMock.replay( config );
     EasyMock.replay( context );
@@ -169,6 +181,8 @@ public class SwitchCaseIdentityAssertionFilterTest {
     EasyMock.expect( config.getInitParameter( "group.principal.case" ) ).andReturn( "none" ).anyTimes();
     ServletContext context = EasyMock.createNiceMock(ServletContext.class);
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     EasyMock.replay( config );
     EasyMock.replay( context );
@@ -195,6 +209,8 @@ public class SwitchCaseIdentityAssertionFilterTest {
     EasyMock.expect( config.getInitParameter( "group.principal.case" ) ).andReturn( "none" ).anyTimes();
     ServletContext context = EasyMock.createNiceMock(ServletContext.class);
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     EasyMock.replay( config );
     EasyMock.replay( context );
@@ -222,6 +238,8 @@ public class SwitchCaseIdentityAssertionFilterTest {
     EasyMock.expect(config.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     ServletContext context = EasyMock.createNiceMock(ServletContext.class);
     EasyMock.expect(config.getServletContext() ).andReturn( context ).anyTimes();
+    EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
+    EasyMock.expect(config.getInitParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
     EasyMock.expect(context.getInitParameter("principal.mapping") ).andReturn( "" ).anyTimes();
     EasyMock.replay( config );
     EasyMock.replay( context );
diff --git a/gateway-provider-security-authz-acls/src/main/java/org/apache/knox/gateway/filter/AclsAuthorizationFilter.java b/gateway-provider-security-authz-acls/src/main/java/org/apache/knox/gateway/filter/AclsAuthorizationFilter.java
index f453a166b..87c50e4b9 100644
--- a/gateway-provider-security-authz-acls/src/main/java/org/apache/knox/gateway/filter/AclsAuthorizationFilter.java
+++ b/gateway-provider-security-authz-acls/src/main/java/org/apache/knox/gateway/filter/AclsAuthorizationFilter.java
@@ -25,8 +25,7 @@ import org.apache.knox.gateway.audit.api.ResourceType;
 import org.apache.knox.gateway.audit.log4j.audit.AuditConstants;
 import org.apache.knox.gateway.i18n.messages.MessagesFactory;
 import org.apache.knox.gateway.security.GroupPrincipal;
-import org.apache.knox.gateway.security.ImpersonatedPrincipal;
-import org.apache.knox.gateway.security.PrimaryPrincipal;
+import org.apache.knox.gateway.security.SubjectUtils;
 
 import javax.security.auth.Subject;
 import javax.servlet.Filter;
@@ -38,7 +37,6 @@ import javax.servlet.ServletResponse;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import java.io.IOException;
-import java.security.AccessController;
 import java.security.Principal;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -114,9 +112,7 @@ public class AclsAuthorizationFilter implements Filter {
     }
   }
 
-  protected boolean enforceAclAuthorizationPolicy(ServletRequest request,
-      ServletResponse response, FilterChain chain) {
-    HttpServletRequest req = (HttpServletRequest) request;
+  protected boolean enforceAclAuthorizationPolicy(ServletRequest request, ServletResponse response, FilterChain chain) {
 
     // before enforcing acls check whether there are no acls defined
     // which would mean that there are no restrictions
@@ -124,23 +120,15 @@ public class AclsAuthorizationFilter implements Filter {
       return true;
     }
 
-    boolean userAccess;
     boolean groupAccess = false;
     boolean ipAddrAccess;
 
-    Subject subject = Subject.getSubject(AccessController.getContext());
-    Principal primaryPrincipal = (Principal)subject.getPrincipals(PrimaryPrincipal.class).toArray()[0];
-    log.primaryPrincipal(primaryPrincipal.getName());
-    Object[] impersonations = subject.getPrincipals(ImpersonatedPrincipal.class).toArray();
-    if (impersonations.length > 0) {
-      log.impersonatedPrincipal(((Principal)impersonations[0]).getName());
-      userAccess = checkUserAcls((Principal)impersonations[0]);
-      log.impersonatedPrincipalHasAccess(userAccess);
-    }
-    else {
-      userAccess = checkUserAcls(primaryPrincipal);
-      log.primaryPrincipalHasAccess(userAccess);
-    }
+    final Subject subject = SubjectUtils.getCurrentSubject();
+    final String effectivePrincipalName = SubjectUtils.getEffectivePrincipalName(subject);
+    log.effectivePrincipal(effectivePrincipalName);
+    boolean userAccess = checkUserAcls(effectivePrincipalName);
+    log.effectivePrincipalHasAccess(userAccess);
+
     Object[] groups = subject.getPrincipals(GroupPrincipal.class).toArray();
     if (groups.length > 0) {
       groupAccess = checkGroupAcls(groups);
@@ -155,8 +143,8 @@ public class AclsAuthorizationFilter implements Filter {
         groupAccess = true;
       }
     }
-    log.remoteIPAddress(req.getRemoteAddr());
-    ipAddrAccess = checkRemoteIpAcls(req.getRemoteAddr());
+    log.remoteIPAddress(((HttpServletRequest) request).getRemoteAddr());
+    ipAddrAccess = checkRemoteIpAcls(((HttpServletRequest) request).getRemoteAddr());
     log.remoteIPAddressHasAccess(ipAddrAccess);
 
     if ("OR".equals(aclProcessingMode)) {
@@ -191,20 +179,20 @@ public class AclsAuthorizationFilter implements Filter {
     return allowed;
   }
 
-  boolean checkUserAcls(Principal user) {
+  boolean checkUserAcls(String userName) {
     boolean allowed = false;
-    if (user == null) {
+    if (userName == null) {
       return false;
     }
     if (parser.anyUser) {
       allowed = true;
     }
     else {
-      if (parser.users.contains(user.getName())) {
+      if (parser.users.contains(userName)) {
         allowed = true;
       }
       else if (parser.users.contains("KNOX_ADMIN_USERS") &&
-          adminUsers.contains(user.getName())) {
+          adminUsers.contains(userName)) {
         allowed = true;
       }
     }
diff --git a/gateway-provider-security-authz-acls/src/main/java/org/apache/knox/gateway/filter/AclsAuthorizationMessages.java b/gateway-provider-security-authz-acls/src/main/java/org/apache/knox/gateway/filter/AclsAuthorizationMessages.java
index 7b48af81c..e630c7d6c 100644
--- a/gateway-provider-security-authz-acls/src/main/java/org/apache/knox/gateway/filter/AclsAuthorizationMessages.java
+++ b/gateway-provider-security-authz-acls/src/main/java/org/apache/knox/gateway/filter/AclsAuthorizationMessages.java
@@ -42,17 +42,11 @@ public interface AclsAuthorizationMessages {
   @Message( level = MessageLevel.DEBUG, text = "Access Granted: {0}" )
   void accessGranted(boolean accessGranted);
 
-  @Message( level = MessageLevel.DEBUG, text = "PrimaryPrincipal: {0}" )
-  void primaryPrincipal(String name);
+  @Message( level = MessageLevel.DEBUG, text = "Effective principal: {0}" )
+  void effectivePrincipal(String name);
 
-  @Message( level = MessageLevel.DEBUG, text = "ImpersonatedPrincipal: {0}" )
-  void impersonatedPrincipal(String name);
-
-  @Message( level = MessageLevel.DEBUG, text = "ImpersonatedPrincipal has access: {0}" )
-  void impersonatedPrincipalHasAccess(boolean userAccess);
-
-  @Message( level = MessageLevel.DEBUG, text = "PrimaryPrincipal has access: {0}" )
-  void primaryPrincipalHasAccess(boolean userAccess);
+  @Message( level = MessageLevel.DEBUG, text = "Effective principal has access: {0}" )
+  void effectivePrincipalHasAccess(boolean userAccess);
 
   @Message( level = MessageLevel.DEBUG, text = "GroupPrincipal has access: {0}" )
   void groupPrincipalHasAccess(boolean groupAccess);
diff --git a/gateway-provider-security-hadoopauth/src/main/java/org/apache/knox/gateway/hadoopauth/filter/HadoopAuthFilter.java b/gateway-provider-security-hadoopauth/src/main/java/org/apache/knox/gateway/hadoopauth/filter/HadoopAuthFilter.java
index edc2d3e88..98e03d1cb 100755
--- a/gateway-provider-security-hadoopauth/src/main/java/org/apache/knox/gateway/hadoopauth/filter/HadoopAuthFilter.java
+++ b/gateway-provider-security-hadoopauth/src/main/java/org/apache/knox/gateway/hadoopauth/filter/HadoopAuthFilter.java
@@ -17,8 +17,6 @@
  */
 package org.apache.knox.gateway.hadoopauth.filter;
 
-import org.apache.hadoop.security.authorize.AuthorizationException;
-import org.apache.hadoop.util.HttpExceptionUtils;
 import org.apache.knox.gateway.GatewayServer;
 import org.apache.knox.gateway.audit.api.Action;
 import org.apache.knox.gateway.audit.api.ActionOutcome;
@@ -29,6 +27,7 @@ import org.apache.knox.gateway.audit.api.Auditor;
 import org.apache.knox.gateway.audit.api.ResourceType;
 import org.apache.knox.gateway.audit.log4j.audit.AuditConstants;
 import org.apache.knox.gateway.config.GatewayConfig;
+import org.apache.knox.gateway.context.ContextAttributes;
 import org.apache.knox.gateway.filter.AbstractGatewayFilter;
 import org.apache.knox.gateway.hadoopauth.HadoopAuthMessages;
 import org.apache.knox.gateway.hadoopauth.deploy.HadoopAuthDeploymentContributor;
@@ -40,6 +39,8 @@ import org.apache.knox.gateway.services.ServiceType;
 import org.apache.knox.gateway.services.security.AliasService;
 import org.apache.knox.gateway.services.security.AliasServiceException;
 import org.apache.knox.gateway.util.AuthFilterUtils;
+import org.apache.knox.gateway.util.AuthorizationException;
+import org.apache.knox.gateway.util.HttpExceptionUtils;
 
 import javax.security.auth.Subject;
 import javax.servlet.FilterChain;
@@ -58,6 +59,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Enumeration;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Locale;
 import java.util.Properties;
 import java.util.Set;
@@ -86,8 +88,6 @@ import static org.apache.knox.gateway.util.AuthFilterUtils.DEFAULT_AUTH_UNAUTHEN
 public class HadoopAuthFilter extends
     org.apache.hadoop.security.authentication.server.AuthenticationFilter {
 
-  private static final String QUERY_PARAMETER_DOAS = "doAs";
-  private static final String PROXYUSER_PREFIX = "hadoop.proxyuser";
   static final String SUPPORT_JWT = "support.jwt";
 
   private static final HadoopAuthMessages LOG = MessagesFactory.get(HadoopAuthMessages.class);
@@ -115,7 +115,9 @@ public class HadoopAuthFilter extends
   @Override
   public void init(FilterConfig filterConfig) throws ServletException {
     this.topologyName = (String) filterConfig.getInitParameter("clusterName");
-    AuthFilterUtils.refreshSuperUserGroupsConfiguration(filterConfig, PROXYUSER_PREFIX, topologyName, HadoopAuthDeploymentContributor.NAME);
+    final List<String> initParameterNames = AuthFilterUtils.getInitParameterNamesAsList(filterConfig);
+    AuthFilterUtils.refreshSuperUserGroupsConfiguration(filterConfig, initParameterNames, topologyName, HadoopAuthDeploymentContributor.NAME);
+    filterConfig.getServletContext().setAttribute(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE, Boolean.TRUE);
 
     Collection<String> ignoredServices = null;
 
@@ -200,7 +202,7 @@ public class HadoopAuthFilter extends
     HttpServletRequest proxyRequest = null;
     final String remoteUser = request.getRemoteUser();
     if (!ignoreDoAs(remoteUser)) {
-      final String doAsUser = request.getParameter(QUERY_PARAMETER_DOAS);
+      final String doAsUser = request.getParameter(AuthFilterUtils.QUERY_PARAMETER_DOAS);
       if (doAsUser != null && !doAsUser.equals(remoteUser)) {
         LOG.hadoopAuthDoAsUser(doAsUser, remoteUser, request.getRemoteAddr());
         if (request.getUserPrincipal() != null) {
@@ -237,8 +239,7 @@ public class HadoopAuthFilter extends
       final Subject sub = new Subject();
       sub.getPrincipals().add(new PrimaryPrincipal("anonymous"));
       LOG.unauthenticatedPathBypass(((HttpServletRequest) request).getRequestURI(), unAuthenticatedPaths.toString());
-      continueWithEstablishedSecurityContext(sub, (HttpServletRequest) request,
-          (HttpServletResponse) response, chain);
+      continueWithEstablishedSecurityContext(sub, (HttpServletRequest) request, (HttpServletResponse) response, chain);
 
     } catch (final Exception e) {
       LOG.unauthenticatedPathError(
diff --git a/gateway-provider-security-hadoopauth/src/main/java/org/apache/knox/gateway/hadoopauth/filter/HadoopAuthPostFilter.java b/gateway-provider-security-hadoopauth/src/main/java/org/apache/knox/gateway/hadoopauth/filter/HadoopAuthPostFilter.java
index e839cf733..c87c445bf 100755
--- a/gateway-provider-security-hadoopauth/src/main/java/org/apache/knox/gateway/hadoopauth/filter/HadoopAuthPostFilter.java
+++ b/gateway-provider-security-hadoopauth/src/main/java/org/apache/knox/gateway/hadoopauth/filter/HadoopAuthPostFilter.java
@@ -37,7 +37,9 @@ import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
 import org.apache.knox.gateway.audit.api.AuditContext;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.knox.gateway.security.ImpersonatedPrincipal;
 import org.apache.knox.gateway.security.PrimaryPrincipal;
 import org.apache.knox.gateway.audit.api.AuditService;
 import org.apache.knox.gateway.audit.api.AuditServiceFactory;
@@ -51,6 +53,7 @@ import org.apache.knox.gateway.audit.api.Action;
 import org.apache.knox.gateway.audit.api.ActionOutcome;
 import org.apache.knox.gateway.audit.api.Auditor;
 import org.apache.knox.gateway.services.security.token.UnknownTokenException;
+import org.apache.knox.gateway.util.AuthFilterUtils;
 
 public class HadoopAuthPostFilter implements Filter {
 
@@ -96,7 +99,19 @@ public class HadoopAuthPostFilter implements Filter {
       final String principal = ((HttpServletRequest) request).getRemoteUser();
       if (principal != null) {
         subject = new Subject();
-        subject.getPrincipals().add(new PrimaryPrincipal(principal));
+        // see org.apache.knox.gateway.util.AuthFilterUtils.getProxyRequest(...).new HttpServletRequestWrapper() {...}.getAttribute(String)
+        final String realUserName = (String) request.getAttribute(AuthFilterUtils.REAL_USER_NAME_ATTRIBUTE);
+        if (StringUtils.isNotBlank(realUserName)) {
+          // this means the 'doAs' query parameter was set and proxyuser authorization went well
+          // in this case, we need to set two principals:
+          // - the primary principal should be the original user who initiated the request
+          // - the impersonated principal should be the 'doAs' user (which)
+          subject.getPrincipals().add(new PrimaryPrincipal(realUserName));
+          subject.getPrincipals().add(new ImpersonatedPrincipal(principal));
+        } else {
+          // simple case: no 'doAs' -> only the primary principal should be carried over (= the original user who initiated the request)
+          subject.getPrincipals().add(new PrimaryPrincipal(principal));
+        }
         AuditContext context = auditService.getContext();
         context.setUsername(principal);
         auditService.attachContext(context);
diff --git a/gateway-provider-security-hadoopauth/src/test/java/org/apache/knox/gateway/hadoopauth/filter/HadoopAuthFilterTest.java b/gateway-provider-security-hadoopauth/src/test/java/org/apache/knox/gateway/hadoopauth/filter/HadoopAuthFilterTest.java
index 0920a56b6..8b1755fd7 100644
--- a/gateway-provider-security-hadoopauth/src/test/java/org/apache/knox/gateway/hadoopauth/filter/HadoopAuthFilterTest.java
+++ b/gateway-provider-security-hadoopauth/src/test/java/org/apache/knox/gateway/hadoopauth/filter/HadoopAuthFilterTest.java
@@ -34,6 +34,7 @@ import static org.junit.Assert.assertTrue;
 
 import org.apache.knox.gateway.GatewayFilter;
 import org.apache.knox.gateway.config.GatewayConfig;
+import org.apache.knox.gateway.context.ContextAttributes;
 import org.apache.knox.gateway.provider.federation.jwt.filter.AbstractJWTFilter;
 import org.apache.knox.gateway.provider.federation.jwt.filter.JWTFederationFilter;
 import org.apache.knox.gateway.provider.federation.jwt.filter.SignatureVerificationCache;
@@ -123,6 +124,8 @@ public class HadoopAuthFilterTest {
     topology.setName("Sample");
 
     ServletContext servletContext = createMock(ServletContext.class);
+    servletContext.setAttribute(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE, Boolean.TRUE);
+    EasyMock.expectLastCall();
     expect(servletContext.getAttribute("signer.secret.provider.object")).andReturn(null).atLeastOnce();
 
     FilterConfig filterConfig = createMock(FilterConfig.class);
@@ -197,7 +200,7 @@ public class HadoopAuthFilterTest {
     HttpServletResponse response = EasyMock.createNiceMock(HttpServletResponse.class);
 
     GatewayFilter.Holder filterConfig = createMock(GatewayFilter.Holder.class);
-    expect(filterConfig.getInitParameterNames()).andReturn(Collections.enumeration(Collections.emptyList()));
+    expect(filterConfig.getInitParameterNames()).andReturn(Collections.enumeration(Collections.emptyList())).anyTimes();
     expect(filterConfig.getInitParameter(GatewayConfig.PROXYUSER_SERVICES_IGNORE_DOAS)).andReturn("service").atLeastOnce();
     expect(filterConfig.getInitParameter("config.prefix")).andReturn("some.prefix").atLeastOnce();
     expect(filterConfig.getInitParameter("support.jwt")).andReturn("false").anyTimes();
@@ -213,6 +216,8 @@ public class HadoopAuthFilterTest {
 
     final ServletContext servletContext = createMock(ServletContext.class);
     expect(servletContext.getAttribute("signer.secret.provider.object")).andReturn(null).atLeastOnce();
+    servletContext.setAttribute(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE, Boolean.TRUE);
+    EasyMock.expectLastCall();
     expect(filterConfig.getServletContext()).andReturn(servletContext).atLeastOnce();
 
     final HadoopAuthFilter hadoopAuthFilter = createMockBuilder(HadoopAuthFilter.class).addMockedMethod("getConfiguration", String.class, FilterConfig.class).withConstructor()
@@ -244,7 +249,7 @@ public class HadoopAuthFilterTest {
     HttpServletRequest request_semicolon = EasyMock.createNiceMock(HttpServletRequest.class);
 
     GatewayFilter.Holder filterConfig = createMock(GatewayFilter.Holder.class);
-    expect(filterConfig.getInitParameterNames()).andReturn(Collections.enumeration(Collections.emptyList()));
+    expect(filterConfig.getInitParameterNames()).andReturn(Collections.enumeration(Collections.emptyList())).anyTimes();
     expect(filterConfig.getInitParameter(GatewayConfig.PROXYUSER_SERVICES_IGNORE_DOAS)).andReturn("service").atLeastOnce();
     expect(filterConfig.getInitParameter("config.prefix")).andReturn("some.prefix").atLeastOnce();
     expect(filterConfig.getInitParameter("support.jwt")).andReturn("false").anyTimes();
@@ -271,6 +276,8 @@ public class HadoopAuthFilterTest {
 
     final ServletContext servletContext = createMock(ServletContext.class);
     expect(servletContext.getAttribute("signer.secret.provider.object")).andReturn(null).atLeastOnce();
+    servletContext.setAttribute(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE, Boolean.TRUE);
+    EasyMock.expectLastCall();
     expect(filterConfig.getServletContext()).andReturn(servletContext).atLeastOnce();
 
     final HadoopAuthFilter hadoopAuthFilter = createMockBuilder(HadoopAuthFilter.class).addMockedMethod("getConfiguration", String.class, FilterConfig.class).withConstructor()
@@ -303,7 +310,7 @@ public class HadoopAuthFilterTest {
     HttpServletRequest request_query = EasyMock.createNiceMock(HttpServletRequest.class);
 
     GatewayFilter.Holder filterConfig = createMock(GatewayFilter.Holder.class);
-    expect(filterConfig.getInitParameterNames()).andReturn(Collections.enumeration(Collections.emptyList()));
+    expect(filterConfig.getInitParameterNames()).andReturn(Collections.enumeration(Collections.emptyList())).anyTimes();
     expect(filterConfig.getInitParameter(GatewayConfig.PROXYUSER_SERVICES_IGNORE_DOAS)).andReturn("service").atLeastOnce();
     expect(filterConfig.getInitParameter("config.prefix")).andReturn("some.prefix").atLeastOnce();
     expect(filterConfig.getInitParameter("support.jwt")).andReturn("false").anyTimes();
@@ -330,6 +337,8 @@ public class HadoopAuthFilterTest {
 
     final ServletContext servletContext = createMock(ServletContext.class);
     expect(servletContext.getAttribute("signer.secret.provider.object")).andReturn(null).atLeastOnce();
+    servletContext.setAttribute(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE, Boolean.TRUE);
+    EasyMock.expectLastCall();
     expect(filterConfig.getServletContext()).andReturn(servletContext).atLeastOnce();
 
     final HadoopAuthFilter hadoopAuthFilter = createMockBuilder(HadoopAuthFilter.class).addMockedMethod("getConfiguration", String.class, FilterConfig.class).withConstructor()
@@ -362,7 +371,7 @@ public class HadoopAuthFilterTest {
     HttpServletRequest request_ampersand = EasyMock.createNiceMock(HttpServletRequest.class);
 
     GatewayFilter.Holder filterConfig = createMock(GatewayFilter.Holder.class);
-    expect(filterConfig.getInitParameterNames()).andReturn(Collections.enumeration(Collections.emptyList()));
+    expect(filterConfig.getInitParameterNames()).andReturn(Collections.enumeration(Collections.emptyList())).anyTimes();
     expect(filterConfig.getInitParameter(GatewayConfig.PROXYUSER_SERVICES_IGNORE_DOAS)).andReturn("service").atLeastOnce();
     expect(filterConfig.getInitParameter("config.prefix")).andReturn("some.prefix").atLeastOnce();
     expect(filterConfig.getInitParameter("support.jwt")).andReturn("false").anyTimes();
@@ -389,6 +398,8 @@ public class HadoopAuthFilterTest {
 
     final ServletContext servletContext = createMock(ServletContext.class);
     expect(servletContext.getAttribute("signer.secret.provider.object")).andReturn(null).atLeastOnce();
+    servletContext.setAttribute(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE, Boolean.TRUE);
+    EasyMock.expectLastCall();
     expect(filterConfig.getServletContext()).andReturn(servletContext).atLeastOnce();
 
     final HadoopAuthFilter hadoopAuthFilter = createMockBuilder(HadoopAuthFilter.class).addMockedMethod("getConfiguration", String.class, FilterConfig.class).withConstructor()
@@ -421,7 +432,7 @@ public class HadoopAuthFilterTest {
     HttpServletRequest request_dash = EasyMock.createNiceMock(HttpServletRequest.class);
 
     GatewayFilter.Holder filterConfig = createMock(GatewayFilter.Holder.class);
-    expect(filterConfig.getInitParameterNames()).andReturn(Collections.enumeration(Collections.emptyList()));
+    expect(filterConfig.getInitParameterNames()).andReturn(Collections.enumeration(Collections.emptyList())).anyTimes();
     expect(filterConfig.getInitParameter(GatewayConfig.PROXYUSER_SERVICES_IGNORE_DOAS)).andReturn("service").atLeastOnce();
     expect(filterConfig.getInitParameter("config.prefix")).andReturn("some.prefix").atLeastOnce();
     expect(filterConfig.getInitParameter("support.jwt")).andReturn("false").anyTimes();
@@ -448,6 +459,8 @@ public class HadoopAuthFilterTest {
 
     final ServletContext servletContext = createMock(ServletContext.class);
     expect(servletContext.getAttribute("signer.secret.provider.object")).andReturn(null).atLeastOnce();
+    servletContext.setAttribute(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE, Boolean.TRUE);
+    EasyMock.expectLastCall();
     expect(filterConfig.getServletContext()).andReturn(servletContext).atLeastOnce();
 
     final HadoopAuthFilter hadoopAuthFilter = createMockBuilder(HadoopAuthFilter.class).addMockedMethod("getConfiguration", String.class, FilterConfig.class).withConstructor()
@@ -481,7 +494,7 @@ public class HadoopAuthFilterTest {
 
 
     GatewayFilter.Holder filterConfig = createMock(GatewayFilter.Holder.class);
-    expect(filterConfig.getInitParameterNames()).andReturn(Collections.enumeration(Collections.emptyList()));
+    expect(filterConfig.getInitParameterNames()).andReturn(Collections.enumeration(Collections.emptyList())).anyTimes();
     expect(filterConfig.getInitParameter(GatewayConfig.PROXYUSER_SERVICES_IGNORE_DOAS)).andReturn("service").atLeastOnce();
     expect(filterConfig.getInitParameter("config.prefix")).andReturn("some.prefix").atLeastOnce();
     expect(filterConfig.getInitParameter("support.jwt")).andReturn("false").anyTimes();
@@ -508,6 +521,8 @@ public class HadoopAuthFilterTest {
 
     final ServletContext servletContext = createMock(ServletContext.class);
     expect(servletContext.getAttribute("signer.secret.provider.object")).andReturn(null).atLeastOnce();
+    servletContext.setAttribute(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE, Boolean.TRUE);
+    EasyMock.expectLastCall();
     expect(filterConfig.getServletContext()).andReturn(servletContext).atLeastOnce();
 
     final HadoopAuthFilter hadoopAuthFilter = createMockBuilder(HadoopAuthFilter.class).addMockedMethod("getConfiguration", String.class, FilterConfig.class).withConstructor()
@@ -556,7 +571,7 @@ public class HadoopAuthFilterTest {
 
   private HadoopAuthFilter testIfJwtSupported(String supportJwt) throws Exception {
     final GatewayFilter.Holder filterConfig = createMock(GatewayFilter.Holder.class);
-    expect(filterConfig.getInitParameterNames()).andReturn(Collections.enumeration(Collections.emptyList()));
+    expect(filterConfig.getInitParameterNames()).andReturn(Collections.enumeration(Collections.emptyList())).anyTimes();
     expect(filterConfig.getInitParameter(GatewayConfig.PROXYUSER_SERVICES_IGNORE_DOAS)).andReturn("service").atLeastOnce();
     expect(filterConfig.getInitParameter("config.prefix")).andReturn("some.prefix").atLeastOnce();
     expect(filterConfig.getInitParameter("support.jwt")).andReturn(supportJwt).anyTimes();
@@ -585,6 +600,8 @@ public class HadoopAuthFilterTest {
       expect(servletContext.getAttribute(GatewayServices.GATEWAY_SERVICES_ATTRIBUTE)).andReturn(null).anyTimes();
     }
     expect(filterConfig.getServletContext()).andReturn(servletContext).atLeastOnce();
+    servletContext.setAttribute(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE, Boolean.TRUE);
+    EasyMock.expectLastCall();
 
     final HadoopAuthFilter hadoopAuthFilter = createMockBuilder(HadoopAuthFilter.class).addMockedMethod("getConfiguration", String.class, FilterConfig.class).withConstructor()
         .createMock();
diff --git a/gateway-provider-security-jwt/src/main/java/org/apache/knox/gateway/provider/federation/jwt/filter/AbstractJWTFilter.java b/gateway-provider-security-jwt/src/main/java/org/apache/knox/gateway/provider/federation/jwt/filter/AbstractJWTFilter.java
index 01412e404..5f0a2a512 100644
--- a/gateway-provider-security-jwt/src/main/java/org/apache/knox/gateway/provider/federation/jwt/filter/AbstractJWTFilter.java
+++ b/gateway-provider-security-jwt/src/main/java/org/apache/knox/gateway/provider/federation/jwt/filter/AbstractJWTFilter.java
@@ -57,6 +57,7 @@ import org.apache.knox.gateway.filter.AbstractGatewayFilter;
 import org.apache.knox.gateway.i18n.messages.MessagesFactory;
 import org.apache.knox.gateway.provider.federation.jwt.JWTMessages;
 import org.apache.knox.gateway.security.PrimaryPrincipal;
+import org.apache.knox.gateway.security.SubjectUtils;
 import org.apache.knox.gateway.services.GatewayServices;
 import org.apache.knox.gateway.services.ServiceLifecycleException;
 import org.apache.knox.gateway.services.ServiceType;
@@ -231,11 +232,13 @@ public abstract class AbstractJWTFilter implements Filter {
     return valid;
   }
 
-  protected void continueWithEstablishedSecurityContext(final Subject subject, final HttpServletRequest request, final HttpServletResponse response, final FilterChain chain) throws IOException, ServletException {
-    Principal principal = (Principal) subject.getPrincipals(PrimaryPrincipal.class).toArray()[0];
+  protected void continueWithEstablishedSecurityContext(final Subject subject,
+                                                        final HttpServletRequest request,
+                                                        final HttpServletResponse response,
+                                                        final FilterChain chain) throws IOException, ServletException {
     AuditContext context = auditService.getContext();
     if (context != null) {
-      context.setUsername( principal.getName() );
+      context.setUsername( SubjectUtils.getPrimaryPrincipalName(subject) );
       auditService.attachContext(context);
       String sourceUri = (String)request.getAttribute( AbstractGatewayFilter.SOURCE_REQUEST_CONTEXT_URL_ATTRIBUTE_NAME );
       if (sourceUri != null) {
diff --git a/gateway-provider-security-jwt/src/main/java/org/apache/knox/gateway/provider/federation/jwt/filter/AccessTokenFederationFilter.java b/gateway-provider-security-jwt/src/main/java/org/apache/knox/gateway/provider/federation/jwt/filter/AccessTokenFederationFilter.java
index 0f427a1ef..3e2b444ef 100644
--- a/gateway-provider-security-jwt/src/main/java/org/apache/knox/gateway/provider/federation/jwt/filter/AccessTokenFederationFilter.java
+++ b/gateway-provider-security-jwt/src/main/java/org/apache/knox/gateway/provider/federation/jwt/filter/AccessTokenFederationFilter.java
@@ -44,7 +44,7 @@ import java.security.Principal;
 import java.security.PrivilegedActionException;
 import java.security.PrivilegedExceptionAction;
 import java.text.ParseException;
-import java.util.HashSet;
+import java.util.Collections;
 import java.util.Locale;
 import java.util.Set;
 
@@ -163,11 +163,7 @@ public class AccessTokenFederationFilter implements Filter {
 
   private Subject createSubjectFromToken(JWTToken token) {
     final String principal = token.getPrincipal();
-
-    HashSet emptySet = new HashSet();
-    Set<Principal> principals = new HashSet<>();
-    Principal p = new PrimaryPrincipal(principal);
-    principals.add(p);
+    final Set<Principal> principals = Collections.singleton(new PrimaryPrincipal(principal));
 
     // The newly constructed Sets check whether this Subject has been set read-only
     // before permitting subsequent modifications. The newly created Sets also prevent
@@ -176,6 +172,6 @@ public class AccessTokenFederationFilter implements Filter {
     // To modify the Principals Set, the caller must have AuthPermission("modifyPrincipals").
     // To modify the public credential Set, the caller must have AuthPermission("modifyPublicCredentials").
     // To modify the private credential Set, the caller must have AuthPermission("modifyPrivateCredentials").
-    return new javax.security.auth.Subject(true, principals, emptySet, emptySet);
+    return new javax.security.auth.Subject(true, principals, Collections.emptySet(), Collections.emptySet());
   }
 }
diff --git a/gateway-provider-security-shiro/src/main/java/org/apache/knox/gateway/filter/ShiroSubjectIdentityAdapter.java b/gateway-provider-security-shiro/src/main/java/org/apache/knox/gateway/filter/ShiroSubjectIdentityAdapter.java
index 7bcbc82f9..4fb945791 100644
--- a/gateway-provider-security-shiro/src/main/java/org/apache/knox/gateway/filter/ShiroSubjectIdentityAdapter.java
+++ b/gateway-provider-security-shiro/src/main/java/org/apache/knox/gateway/filter/ShiroSubjectIdentityAdapter.java
@@ -20,6 +20,7 @@ package org.apache.knox.gateway.filter;
 import java.io.IOException;
 import java.security.Principal;
 import java.security.PrivilegedExceptionAction;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.Set;
 import java.util.concurrent.Callable;
@@ -83,6 +84,7 @@ public class ShiroSubjectIdentityAdapter implements Filter {
       this.chain = chain;
     }
 
+    @SuppressWarnings("unchecked")
     @Override
     public Void call() throws Exception {
       PrivilegedExceptionAction<Void> action = new PrivilegedExceptionAction<Void>() {
@@ -99,7 +101,6 @@ public class ShiroSubjectIdentityAdapter implements Filter {
       }
 
       final String principal = shiroSubject.getPrincipal().toString();
-      HashSet emptySet = new HashSet();
       Set<Principal> principals = new HashSet<>();
       Principal p = new PrimaryPrincipal(principal);
       principals.add(p);
@@ -141,7 +142,7 @@ public class ShiroSubjectIdentityAdapter implements Filter {
       // To modify the Principals Set, the caller must have AuthPermission("modifyPrincipals").
       // To modify the public credential Set, the caller must have AuthPermission("modifyPublicCredentials").
       // To modify the private credential Set, the caller must have AuthPermission("modifyPrivateCredentials").
-      javax.security.auth.Subject subject = new javax.security.auth.Subject(true, principals, emptySet, emptySet);
+      javax.security.auth.Subject subject = new javax.security.auth.Subject(true, principals, Collections.emptySet(), Collections.emptySet());
       javax.security.auth.Subject.doAs( subject, action );
 
       return null;
diff --git a/gateway-release/home/conf/topologies/homepage.xml b/gateway-release/home/conf/topologies/homepage.xml
index 336d130d2..530237221 100644
--- a/gateway-release/home/conf/topologies/homepage.xml
+++ b/gateway-release/home/conf/topologies/homepage.xml
@@ -60,6 +60,22 @@
          <role>identity-assertion</role>
          <name>Default</name>
          <enabled>true</enabled>
+         <param>
+            <name>haddop.proxyuser.impersonation.enabled</name>
+            <value>false</value>
+         </param>
+         <param>
+            <name>haddop.proxyuser.admin.users</name>
+            <value>*</value>
+         </param>
+         <param>
+            <name>hadoop.proxyuser.admin.groups</name>
+            <value>*</value>
+         </param>
+         <param>
+            <name>hadoop.proxyuser.admin.hosts</name>
+            <value>*</value>
+         </param>
       </provider>
    </gateway>
    <service>
@@ -97,18 +113,6 @@
           <name>knox.token.type</name>
           <value>JWT</value>
       </param>
-      <param>
-          <name>knox.token.proxyuser.admin.users</name>
-          <value>*</value>
-      </param>
-      <param>
-          <name>knox.token.proxyuser.admin.groups</name>
-          <value>*</value>
-      </param>
-      <param>
-          <name>knox.token.proxyuser.admin.hosts</name>
-          <value>*</value>
-      </param>
    </service>
    <application>
       <name>token-generation</name>
diff --git a/gateway-service-knoxtoken/src/main/java/org/apache/knox/gateway/service/knoxtoken/TokenResource.java b/gateway-service-knoxtoken/src/main/java/org/apache/knox/gateway/service/knoxtoken/TokenResource.java
index ac3d5d0b1..7b769e792 100644
--- a/gateway-service-knoxtoken/src/main/java/org/apache/knox/gateway/service/knoxtoken/TokenResource.java
+++ b/gateway-service-knoxtoken/src/main/java/org/apache/knox/gateway/service/knoxtoken/TokenResource.java
@@ -61,12 +61,11 @@ import com.nimbusds.jose.crypto.MACSigner;
 import com.nimbusds.jose.util.ByteUtils;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.knox.gateway.config.GatewayConfig;
+import org.apache.knox.gateway.context.ContextAttributes;
 import org.apache.knox.gateway.i18n.messages.MessagesFactory;
 import org.apache.knox.gateway.security.GroupPrincipal;
 import org.apache.knox.gateway.security.SubjectUtils;
-import org.apache.knox.gateway.service.knoxtoken.deploy.TokenServiceDeploymentContributor;
 import org.apache.knox.gateway.services.ServiceType;
 import org.apache.knox.gateway.services.GatewayServices;
 import org.apache.knox.gateway.services.ServiceLifecycleException;
@@ -87,7 +86,6 @@ import org.apache.knox.gateway.services.security.token.UnknownTokenException;
 import org.apache.knox.gateway.services.security.token.impl.JWT;
 import org.apache.knox.gateway.services.security.token.impl.JWTToken;
 import org.apache.knox.gateway.services.security.token.impl.TokenMAC;
-import org.apache.knox.gateway.util.AuthFilterUtils;
 import org.apache.knox.gateway.util.JsonUtils;
 import org.apache.knox.gateway.util.Tokens;
 
@@ -143,8 +141,6 @@ public class TokenResource {
   static final String DISABLE_PATH = "/disable";
   private static final String TARGET_ENDPOINT_PULIC_CERT_PEM = TOKEN_PARAM_PREFIX + "target.endpoint.cert.pem";
   static final String QUERY_PARAMETER_DOAS = "doAs";
-  static final String PROXYUSER_PREFIX = TOKEN_PARAM_PREFIX + "proxyuser";
-  static final String IMPERSONATION_ENABLED_PARAM = TOKEN_PARAM_PREFIX + "impersonation.enabled";
   private static final String IMPERSONATION_ENABLED_TEXT = "impersonationEnabled";
   public static final String KNOX_TOKEN_INCLUDE_GROUPS = TOKEN_PARAM_PREFIX + "include.groups";
   public static final String KNOX_TOKEN_ISSUER = TOKEN_PARAM_PREFIX + "issuer";
@@ -172,7 +168,6 @@ public class TokenResource {
   private int tokenLimitPerUser;
   private boolean includeGroupsInTokenAllowed;
   private String tokenIssuer;
-  private boolean impersonationEnabled;
 
   enum UserLimitExceededAction {REMOVE_OLDEST, RETURN_ERROR};
   private UserLimitExceededAction userLimitExceededAction = UserLimitExceededAction.RETURN_ERROR;
@@ -269,12 +264,6 @@ public class TokenResource {
       endpointPublicCert = targetEndpointPublicCert;
     }
 
-    // KnoxToken impersonation should be configurable regardless of the token state
-    // management status (i.e. even if token state management is enabled users
-    // should be able to opt-out token impersonation
-    final String impersonationEnabledValue = context.getInitParameter(IMPERSONATION_ENABLED_PARAM);
-    impersonationEnabled = impersonationEnabledValue == null ? Boolean.FALSE : Boolean.parseBoolean(impersonationEnabledValue);
-
     // If server-managed token expiration is configured, set the token state service
     if (isServerManagedTokenStateEnabled()) {
       String topologyName = getTopologyName();
@@ -320,12 +309,6 @@ public class TokenResource {
       } else {
         log.noRenewersConfigured(topologyName);
       }
-
-      // refreshing Hadoop ProxyUser groups config only makes sense if token state management is turned on
-      // and impersonation is enabled
-      if (impersonationEnabled) {
-        AuthFilterUtils.refreshSuperUserGroupsConfiguration(context, PROXYUSER_PREFIX, getTopologyName(), TokenServiceDeploymentContributor.ROLE);
-      }
     }
     setTokenStateServiceStatusMap();
   }
@@ -377,7 +360,8 @@ public class TokenResource {
     final Boolean lifespanInputEnabled = lifespanInputEnabledValue == null ? Boolean.TRUE : Boolean.parseBoolean(lifespanInputEnabledValue);
     tokenStateServiceStatusMap.put(LIFESPAN_INPUT_ENABLED_TEXT, lifespanInputEnabled.toString());
 
-    tokenStateServiceStatusMap.put(IMPERSONATION_ENABLED_TEXT, Boolean.toString(impersonationEnabled));
+    final Boolean impersonationEnabled = (Boolean) context.getAttribute(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE);
+    tokenStateServiceStatusMap.put(IMPERSONATION_ENABLED_TEXT, impersonationEnabled == null ? Boolean.FALSE.toString() : impersonationEnabled.toString());
   }
 
   private void populateAllowedTokenStateBackendForTokenGenApp(final String actualTokenServiceName) {
@@ -718,20 +702,17 @@ public class TokenResource {
 
     String userName = request.getUserPrincipal().getName();
     String createdBy = null;
-    // checking the doAs user only makes sense if tokens are managed (this is where we store the userName information)
-    // and if impersonation is enabled
-    if (impersonationEnabled && tokenStateService != null) {
-      final String doAsUser = request.getParameter(QUERY_PARAMETER_DOAS);
-      if (doAsUser != null && !doAsUser.equals(userName)) {
-        try {
-          //this call will authorize the doAs request
-          AuthFilterUtils.authorizeImpersonationRequest(request, doAsUser, getTopologyName(), TokenServiceDeploymentContributor.ROLE);
-          createdBy = userName;
-          userName = doAsUser;
-          log.tokenImpersonationSuccess(createdBy, doAsUser);
-        } catch (AuthorizationException e) {
-          log.tokenImpersonationFailed(e);
-          return Response.status(Response.Status.FORBIDDEN).entity("{ \"" + e.getMessage() + "\" }").build();
+    // checking the doAs user only makes sense if tokens are managed (this is where we store the userName/createdBy information)
+    // and if impersonation was enabled before (on HadoopAuth or identity-assertion level) so the the current subject has at least one ImpersonatedPrincipal principal
+    if (tokenStateService != null) {
+      final Subject subject = SubjectUtils.getCurrentSubject();
+      if (subject != null && SubjectUtils.isImpersonating(subject)) {
+        String primaryPrincipalName = SubjectUtils.getPrimaryPrincipalName(subject);
+        String impersonatedPrincipalName = SubjectUtils.getImpersonatedPrincipalName(subject);
+        if (!primaryPrincipalName.equals(impersonatedPrincipalName)) {
+          createdBy = primaryPrincipalName;
+          userName = impersonatedPrincipalName;
+          log.tokenImpersonationSuccess(createdBy, userName);
         }
       }
     }
diff --git a/gateway-service-knoxtoken/src/test/java/org/apache/knox/gateway/service/knoxtoken/TokenServiceResourceTest.java b/gateway-service-knoxtoken/src/test/java/org/apache/knox/gateway/service/knoxtoken/TokenServiceResourceTest.java
index e61c93ad3..5f8bb0e2e 100644
--- a/gateway-service-knoxtoken/src/test/java/org/apache/knox/gateway/service/knoxtoken/TokenServiceResourceTest.java
+++ b/gateway-service-knoxtoken/src/test/java/org/apache/knox/gateway/service/knoxtoken/TokenServiceResourceTest.java
@@ -82,6 +82,8 @@ import com.nimbusds.jose.crypto.RSASSAVerifier;
 import org.apache.commons.codec.digest.HmacAlgorithms;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.knox.gateway.config.GatewayConfig;
+import org.apache.knox.gateway.context.ContextAttributes;
+import org.apache.knox.gateway.security.ImpersonatedPrincipal;
 import org.apache.knox.gateway.security.PrimaryPrincipal;
 import org.apache.knox.gateway.services.GatewayServices;
 import org.apache.knox.gateway.services.ServiceLifecycleException;
@@ -99,6 +101,7 @@ import org.apache.knox.gateway.services.security.token.impl.JWT;
 import org.apache.knox.gateway.services.security.token.impl.JWTToken;
 import org.apache.knox.gateway.services.security.token.impl.TokenMAC;
 import org.apache.knox.gateway.services.token.impl.JDBCTokenStateService;
+import org.apache.knox.gateway.util.AuthFilterUtils;
 import org.apache.knox.gateway.util.JsonUtils;
 import org.easymock.EasyMock;
 import org.junit.Assert;
@@ -156,6 +159,10 @@ public class TokenServiceResourceTest {
     contextExpectations.forEach((key, value) -> EasyMock.expect(context.getInitParameter(key)).andReturn(value).anyTimes());
     EasyMock.expect(context.getInitParameterNames()).andReturn(Collections.enumeration(contextExpectations.keySet())).anyTimes();
     EasyMock.expect(context.getAttribute("org.apache.knox.gateway.gateway.cluster")).andReturn("topology1").anyTimes();
+    if (contextExpectations.containsKey(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE)) {
+       EasyMock.expect(context.getAttribute(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE)).andReturn(Boolean.parseBoolean(contextExpectations.get(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE))).anyTimes();
+     }
+
     request = EasyMock.createNiceMock(HttpServletRequest.class);
     EasyMock.expect(request.getServletContext()).andReturn(context).anyTimes();
     Principal principal = EasyMock.createNiceMock(Principal.class);
@@ -171,9 +178,6 @@ public class TokenServiceResourceTest {
     if (contextExpectations.containsKey(TokenResource.QUERY_PARAMETER_DOAS)) {
       EasyMock.expect(request.getParameter(TokenResource.QUERY_PARAMETER_DOAS)).andReturn(contextExpectations.get(TokenResource.QUERY_PARAMETER_DOAS)).anyTimes();
     }
-    if (contextExpectations.containsKey(TokenResource.IMPERSONATION_ENABLED_PARAM)) {
-      EasyMock.expect(request.getParameter(TokenResource.IMPERSONATION_ENABLED_PARAM)).andReturn(contextExpectations.get(TokenResource.IMPERSONATION_ENABLED_PARAM)).anyTimes();
-    }
     EasyMock.expect(request.getParameterNames()).andReturn(Collections.emptyEnumeration()).anyTimes();
 
     GatewayServices services = EasyMock.createNiceMock(GatewayServices.class);
@@ -1124,9 +1128,9 @@ public class TokenServiceResourceTest {
     final String impersonatedUser = "testUser";
     final Map<String, String> contextExpectations = new HashMap<>();
     contextExpectations.put(TokenResource.QUERY_PARAMETER_DOAS, impersonatedUser);
-    contextExpectations.put(TokenResource.PROXYUSER_PREFIX + "." + USER_NAME + ".users", impersonatedUser);
-    contextExpectations.put(TokenResource.PROXYUSER_PREFIX + "." + USER_NAME + ".hosts", "*");
-    contextExpectations.put(TokenResource.IMPERSONATION_ENABLED_PARAM, Boolean.toString(enableImpersonation));
+    contextExpectations.put(AuthFilterUtils.PROXYUSER_PREFIX + "." + USER_NAME + ".users", impersonatedUser);
+    contextExpectations.put(AuthFilterUtils.PROXYUSER_PREFIX + "." + USER_NAME + ".hosts", "*");
+    contextExpectations.put(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE, Boolean.toString(enableImpersonation));
     configureCommonExpectations(contextExpectations, Boolean.TRUE);
 
     final TokenResource tr = new TokenResource();
@@ -1134,7 +1138,11 @@ public class TokenServiceResourceTest {
     tr.context = context;
     tr.init();
 
-    tr.doGet();
+    final Subject subject = createTestSubject(USER_NAME);
+    if (enableImpersonation) {
+      subject.getPrincipals().add(new ImpersonatedPrincipal(impersonatedUser));
+    }
+    Subject.doAs(subject,  (PrivilegedAction<Response>) () -> tr.doGet());
 
     final Response getKnoxTokensResponse = getUserTokensResponse(tr, enableImpersonation);
     final Collection<LinkedHashMap<String, Object>> tokens = ((Map<String, Collection<LinkedHashMap<String, Object>>>) JsonUtils
@@ -1463,10 +1471,10 @@ public class TokenServiceResourceTest {
     contextExpectations.put("knox.token.renewer.whitelist", renewers);
 
     if (StringUtils.isNotBlank(impersonatedUser)) {
-      contextExpectations.put(TokenResource.IMPERSONATION_ENABLED_PARAM, "true");
+      contextExpectations.put(ContextAttributes.IMPERSONATION_ENABLED_ATTRIBUTE, "true");
       contextExpectations.put(TokenResource.QUERY_PARAMETER_DOAS, impersonatedUser);
-      contextExpectations.put(TokenResource.PROXYUSER_PREFIX + "." + USER_NAME + ".users", impersonatedUser);
-      contextExpectations.put(TokenResource.PROXYUSER_PREFIX + "." + USER_NAME + ".hosts", "*");
+      contextExpectations.put(AuthFilterUtils.PROXYUSER_PREFIX + "." + USER_NAME + ".users", impersonatedUser);
+      contextExpectations.put(AuthFilterUtils.PROXYUSER_PREFIX + "." + USER_NAME + ".hosts", "*");
     }
 
     configureCommonExpectations(contextExpectations, gatewayLevelConfig);
diff --git a/gateway-spi/pom.xml b/gateway-spi/pom.xml
index 4e29530a8..f73fef1b8 100644
--- a/gateway-spi/pom.xml
+++ b/gateway-spi/pom.xml
@@ -118,6 +118,10 @@
             <groupId>com.fasterxml.jackson.core</groupId>
             <artifactId>jackson-annotations</artifactId>
         </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+        </dependency>
         <dependency>
             <groupId>com.jayway.jsonpath</groupId>
             <artifactId>json-path</artifactId>
diff --git a/gateway-spi/src/main/java/org/apache/knox/gateway/filter/security/AbstractIdentityAssertionBase.java b/gateway-spi/src/main/java/org/apache/knox/gateway/context/ContextAttributes.java
similarity index 62%
copy from gateway-spi/src/main/java/org/apache/knox/gateway/filter/security/AbstractIdentityAssertionBase.java
copy to gateway-spi/src/main/java/org/apache/knox/gateway/context/ContextAttributes.java
index 263a83bae..a54c38937 100644
--- a/gateway-spi/src/main/java/org/apache/knox/gateway/filter/security/AbstractIdentityAssertionBase.java
+++ b/gateway-spi/src/main/java/org/apache/knox/gateway/context/ContextAttributes.java
@@ -15,22 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.knox.gateway.filter.security;
+package org.apache.knox.gateway.context;
 
-import org.apache.knox.gateway.security.SubjectUtils;
+public interface ContextAttributes {
 
-import javax.security.auth.Subject;
-
-public class AbstractIdentityAssertionBase {
-
-  /**
-   * Retrieve the principal to represent the asserted identity from
-   * the provided Subject.
-   * @param subject subject to get the principal from
-   * @return principalName
-   */
-  protected String getPrincipalName(Subject subject) {
-    return SubjectUtils.getPrimaryPrincipalName(subject);
-  }
+  String IMPERSONATION_ENABLED_ATTRIBUTE = "org.apache.knox.gateway.gateway.proxyuser.impersonation.enabled";
 
 }
diff --git a/gateway-spi/src/main/java/org/apache/knox/gateway/filter/security/AbstractIdentityAssertionBase.java b/gateway-spi/src/main/java/org/apache/knox/gateway/filter/security/AbstractIdentityAssertionBase.java
index 263a83bae..0ee186395 100644
--- a/gateway-spi/src/main/java/org/apache/knox/gateway/filter/security/AbstractIdentityAssertionBase.java
+++ b/gateway-spi/src/main/java/org/apache/knox/gateway/filter/security/AbstractIdentityAssertionBase.java
@@ -30,7 +30,7 @@ public class AbstractIdentityAssertionBase {
    * @return principalName
    */
   protected String getPrincipalName(Subject subject) {
-    return SubjectUtils.getPrimaryPrincipalName(subject);
+    return SubjectUtils.getEffectivePrincipalName(subject);
   }
 
 }
diff --git a/gateway-spi/src/main/java/org/apache/knox/gateway/security/SubjectUtils.java b/gateway-spi/src/main/java/org/apache/knox/gateway/security/SubjectUtils.java
index 3e8b41c72..5f7d670ff 100644
--- a/gateway-spi/src/main/java/org/apache/knox/gateway/security/SubjectUtils.java
+++ b/gateway-spi/src/main/java/org/apache/knox/gateway/security/SubjectUtils.java
@@ -28,6 +28,7 @@ import java.util.Set;
  * General utility methods for interrogating the standard java Subject
  */
 public class SubjectUtils {
+
   public static Subject getCurrentSubject() {
     return Subject.getSubject( AccessController.getContext() );
   }
@@ -76,15 +77,8 @@ public class SubjectUtils {
   }
 
   public static String getCurrentEffectivePrincipalName() {
-    String name = null;
-    Subject subject = getCurrentSubject();
-    if( subject != null ) {
-      name = getImpersonatedPrincipalName( subject );
-      if (name == null) {
-        name = getPrimaryPrincipalName(subject);
-      }
-    }
-    return name;
+    final Subject subject = getCurrentSubject();
+    return subject == null ? null : getEffectivePrincipalName(subject);
   }
 
   public static Set<GroupPrincipal> getGroupPrincipals(Subject subject) {
diff --git a/gateway-spi/src/main/java/org/apache/knox/gateway/util/AuthFilterUtils.java b/gateway-spi/src/main/java/org/apache/knox/gateway/util/AuthFilterUtils.java
index 08b82559f..5bd8a7e28 100644
--- a/gateway-spi/src/main/java/org/apache/knox/gateway/util/AuthFilterUtils.java
+++ b/gateway-spi/src/main/java/org/apache/knox/gateway/util/AuthFilterUtils.java
@@ -19,7 +19,7 @@ package org.apache.knox.gateway.util;
 
 import java.security.Principal;
 import java.util.Collections;
-import java.util.Enumeration;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.StringTokenizer;
@@ -36,7 +36,6 @@ import javax.servlet.http.HttpServletRequestWrapper;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.authorize.DefaultImpersonationProvider;
 import org.apache.hadoop.security.authorize.ImpersonationProvider;
 import org.apache.knox.gateway.i18n.GatewaySpiMessages;
@@ -44,6 +43,9 @@ import org.apache.knox.gateway.i18n.messages.MessagesFactory;
 
 public class AuthFilterUtils {
   public static final String DEFAULT_AUTH_UNAUTHENTICATED_PATHS_PARAM = "/knoxtoken/api/v1/jwks.json";
+  public static final String PROXYUSER_PREFIX = "hadoop.proxyuser";
+  public static final String QUERY_PARAMETER_DOAS = "doAs";
+  public static final String REAL_USER_NAME_ATTRIBUTE = "real.user.name";
 
   private static final GatewaySpiMessages LOG = MessagesFactory.get(GatewaySpiMessages.class);
   private static final Map<String, Map<String, ImpersonationProvider>> TOPOLOGY_IMPERSONATION_PROVIDERS = new ConcurrentHashMap<>();
@@ -90,43 +92,39 @@ public class AuthFilterUtils {
     }
   }
 
-  public static void refreshSuperUserGroupsConfiguration(ServletContext context, String prefix, String topologyName, String role) {
+  public static void refreshSuperUserGroupsConfiguration(ServletContext context, List<String> initParameterNames, String topologyName, String role) {
     if (context == null) {
       throw new IllegalArgumentException("Cannot get proxyuser configuration from NULL context");
     }
-    refreshSuperUserGroupsConfiguration(context, null, prefix, topologyName, role);
+    refreshSuperUserGroupsConfiguration(context, null, initParameterNames, topologyName, role);
   }
 
-  public static void refreshSuperUserGroupsConfiguration(FilterConfig filterConfig, String prefix, String topologyName, String role) {
+  public static void refreshSuperUserGroupsConfiguration(FilterConfig filterConfig, List<String> initParameterNames, String topologyName, String role) {
     if (filterConfig == null) {
       throw new IllegalArgumentException("Cannot get proxyuser configuration from NULL filter config");
     }
-    refreshSuperUserGroupsConfiguration(null, filterConfig, prefix, topologyName, role);
+    refreshSuperUserGroupsConfiguration(null, filterConfig, initParameterNames, topologyName, role);
   }
 
-  private static void refreshSuperUserGroupsConfiguration(ServletContext context, FilterConfig filterConfig, String prefix, String topologyName, String role) {
+  private static void refreshSuperUserGroupsConfiguration(ServletContext context, FilterConfig filterConfig, List<String> initParameterNames, String topologyName, String role) {
     final Configuration conf = new Configuration(false);
-    final Enumeration<?> names = context == null ? filterConfig.getInitParameterNames() : context.getInitParameterNames();
-    if (names != null) {
-      while (names.hasMoreElements()) {
-        String name = (String) names.nextElement();
-        if (name.startsWith(prefix + ".")) {
-          String value = context == null ? filterConfig.getInitParameter(name) : context.getInitParameter(name);
-          conf.set(name, value);
-        }
-      }
+    if (initParameterNames != null) {
+      initParameterNames.stream().filter(name -> name.startsWith(PROXYUSER_PREFIX + ".")).forEach(name -> {
+        String value = context == null ? filterConfig.getInitParameter(name) : context.getInitParameter(name);
+        conf.set(name, value);
+      });
     }
 
-    saveImpersonationProvider(prefix, topologyName, role, conf);
+    saveImpersonationProvider(topologyName, role, conf);
   }
 
-  private static void saveImpersonationProvider(String prefix, String topologyName, String role, final Configuration conf) {
+  private static void saveImpersonationProvider(String topologyName, String role, final Configuration conf) {
     refreshSuperUserGroupsLock.lock();
     try {
       final ImpersonationProvider impersonationProvider = new DefaultImpersonationProvider();
       impersonationProvider.setConf(conf);
-      impersonationProvider.init(prefix);
-      LOG.createImpersonationProvider(topologyName, role, prefix, conf.getPropsWithPrefix(prefix + ".").toString());
+      impersonationProvider.init(PROXYUSER_PREFIX);
+      LOG.createImpersonationProvider(topologyName, role, PROXYUSER_PREFIX, conf.getPropsWithPrefix(PROXYUSER_PREFIX + ".").toString());
       TOPOLOGY_IMPERSONATION_PROVIDERS.putIfAbsent(topologyName, new ConcurrentHashMap<String, ImpersonationProvider>());
       TOPOLOGY_IMPERSONATION_PROVIDERS.get(topologyName).put(role, impersonationProvider);
     } finally {
@@ -135,7 +133,11 @@ public class AuthFilterUtils {
   }
 
   public static HttpServletRequest getProxyRequest(HttpServletRequest request, String doAsUser, String topologyName, String role) throws AuthorizationException {
-    final UserGroupInformation remoteRequestUgi = getRemoteRequestUgi(request, doAsUser);
+    return getProxyRequest(request, request.getUserPrincipal().getName(), doAsUser, topologyName, role);
+  }
+
+  public static HttpServletRequest getProxyRequest(HttpServletRequest request, String remoteUser, String doAsUser, String topologyName, String role) throws AuthorizationException {
+    final UserGroupInformation remoteRequestUgi = getRemoteRequestUgi(remoteUser, doAsUser);
     if (remoteRequestUgi != null) {
       authorizeImpersonationRequest(request, remoteRequestUgi, topologyName, role);
 
@@ -149,14 +151,23 @@ public class AuthFilterUtils {
         public Principal getUserPrincipal() {
           return remoteRequestUgi::getUserName;
         }
+
+        @Override
+        public Object getAttribute(String name) {
+          if (name != null && name.equals(REAL_USER_NAME_ATTRIBUTE)) {
+            return remoteRequestUgi.getRealUser().getShortUserName();
+          } else {
+            return super.getAttribute(name);
+          }
+        }
       };
 
     }
     return null;
   }
 
-  public static void authorizeImpersonationRequest(HttpServletRequest request, String doAsUser, String topologyName, String role) throws AuthorizationException {
-    final UserGroupInformation remoteRequestUgi = getRemoteRequestUgi(request, doAsUser);
+  public static void authorizeImpersonationRequest(HttpServletRequest request, String remoteUser, String doAsUser, String topologyName, String role) throws AuthorizationException {
+    final UserGroupInformation remoteRequestUgi = getRemoteRequestUgi(remoteUser, doAsUser);
     if (remoteRequestUgi != null) {
       authorizeImpersonationRequest(request, remoteRequestUgi, topologyName, role);
     }
@@ -168,7 +179,11 @@ public class AuthFilterUtils {
     final ImpersonationProvider impersonationProvider = getImpersonationProvider(topologyName, role);
 
     if (impersonationProvider != null) {
-      impersonationProvider.authorize(remoteRequestUgi, request.getRemoteAddr());
+      try {
+        impersonationProvider.authorize(remoteRequestUgi, request.getRemoteAddr());
+      } catch (org.apache.hadoop.security.authorize.AuthorizationException e) {
+        throw new AuthorizationException(e);
+      }
     } else {
       throw new AuthorizationException("ImpersonationProvider for " + topologyName + " / " + role + " not found!");
     }
@@ -185,13 +200,45 @@ public class AuthFilterUtils {
     return impersonationProvider;
   }
 
-  private static UserGroupInformation getRemoteRequestUgi(HttpServletRequest request, String doAsUser) {
-    if (request.getUserPrincipal() != null) {
-      final String remoteUser = request.getUserPrincipal().getName();
+  private static UserGroupInformation getRemoteRequestUgi(String remoteUser, String doAsUser) {
+    if (remoteUser != null) {
       final UserGroupInformation remoteUserUgi = UserGroupInformation.createRemoteUser(remoteUser);
       return UserGroupInformation.createProxyUser(doAsUser, remoteUserUgi);
     }
     return null;
   }
 
+  public static boolean hasProxyConfig(String topologyName, String role) {
+    return getImpersonationProvider(topologyName, role) != null;
+  }
+
+  public static void removeProxyUserConfig(String topologyName, String role) {
+    if (hasProxyConfig(topologyName, role)) {
+      refreshSuperUserGroupsLock.lock();
+      try {
+        TOPOLOGY_IMPERSONATION_PROVIDERS.get(topologyName).remove(role);
+      } finally {
+        refreshSuperUserGroupsLock.unlock();
+      }
+    }
+  }
+
+  /**
+   * FilterConfig.getInitParameters() returns an enumeration and the first time we
+   * iterate thru on its elements we can process the parameter names as desired
+   * (because hasMoreElements returns true). The subsequent calls, however, will not
+   * succeed because getInitParameters() returns the same object where the
+   * hasMoreElements returns false.
+   * <p>
+   * In classes where there are multiple iterations should be conducted, a
+   * collection should be used instead.
+   *
+   * @return the names of the filter's initialization parameters as a List of
+   *         String objects, or an empty List if the filter has no initialization
+   *         parameters.
+   */
+  public static List<String> getInitParameterNamesAsList(FilterConfig filterConfig) {
+    return filterConfig.getInitParameterNames() == null ? Collections.emptyList() : Collections.list(filterConfig.getInitParameterNames());
+  }
+
 }
diff --git a/gateway-spi/src/main/java/org/apache/knox/gateway/filter/security/AbstractIdentityAssertionBase.java b/gateway-spi/src/main/java/org/apache/knox/gateway/util/AuthorizationException.java
similarity index 62%
copy from gateway-spi/src/main/java/org/apache/knox/gateway/filter/security/AbstractIdentityAssertionBase.java
copy to gateway-spi/src/main/java/org/apache/knox/gateway/util/AuthorizationException.java
index 263a83bae..05a5d20aa 100644
--- a/gateway-spi/src/main/java/org/apache/knox/gateway/filter/security/AbstractIdentityAssertionBase.java
+++ b/gateway-spi/src/main/java/org/apache/knox/gateway/util/AuthorizationException.java
@@ -15,22 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.knox.gateway.filter.security;
+package org.apache.knox.gateway.util;
 
-import org.apache.knox.gateway.security.SubjectUtils;
+import java.io.IOException;
 
-import javax.security.auth.Subject;
+@SuppressWarnings("serial")
+public class AuthorizationException extends IOException {
 
-public class AbstractIdentityAssertionBase {
+  public AuthorizationException(String message) {
+    super(message);
+  }
 
-  /**
-   * Retrieve the principal to represent the asserted identity from
-   * the provided Subject.
-   * @param subject subject to get the principal from
-   * @return principalName
-   */
-  protected String getPrincipalName(Subject subject) {
-    return SubjectUtils.getPrimaryPrincipalName(subject);
+  public AuthorizationException(Throwable e) {
+    super(e);
   }
 
 }
diff --git a/gateway-spi/src/main/java/org/apache/knox/gateway/util/HttpExceptionUtils.java b/gateway-spi/src/main/java/org/apache/knox/gateway/util/HttpExceptionUtils.java
new file mode 100644
index 000000000..9d3979b55
--- /dev/null
+++ b/gateway-spi/src/main/java/org/apache/knox/gateway/util/HttpExceptionUtils.java
@@ -0,0 +1,77 @@
+/*
+ * 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.knox.gateway.util;
+
+import java.io.IOException;
+import java.io.Writer;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import javax.servlet.http.HttpServletResponse;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+
+/**
+ * This class is a partial match of org.apache.hadoop.util.HttpExceptionUtils to
+ * reduce the need for hadoop-common dependency just for creating an error
+ * response in Knox.
+ */
+public class HttpExceptionUtils {
+
+  private static final String ERROR_JSON = "RemoteException";
+  private static final String ERROR_EXCEPTION_JSON = "exception";
+  private static final String ERROR_CLASSNAME_JSON = "javaClassName";
+  private static final String ERROR_MESSAGE_JSON = "message";
+  private static final String APPLICATION_JSON_MIME = "application/json";
+  private static final String ENTER = System.getProperty("line.separator");
+  private static final ObjectWriter WRITER = new ObjectMapper().writerWithDefaultPrettyPrinter();
+
+  /**
+   * Creates a HTTP servlet response serializing the exception in it as JSON.
+   *
+   * @param response the servlet response
+   * @param status   the error code to set in the response
+   * @param ex       the exception to serialize in the response
+   * @throws IOException thrown if there was an error while creating the response
+   */
+  public static void createServletExceptionResponse(HttpServletResponse response, int status, Throwable ex) throws IOException {
+    response.setStatus(status);
+    response.setContentType(APPLICATION_JSON_MIME);
+    final Map<String, Object> json = new LinkedHashMap<>();
+    json.put(ERROR_MESSAGE_JSON, getOneLineMessage(ex));
+    json.put(ERROR_EXCEPTION_JSON, ex.getClass().getSimpleName());
+    json.put(ERROR_CLASSNAME_JSON, ex.getClass().getName());
+    final Map<String, Object> jsonResponse = new LinkedHashMap<>();
+    jsonResponse.put(ERROR_JSON, json);
+    final Writer responseWriter = response.getWriter();
+    WRITER.writeValue(response.getWriter(), jsonResponse);
+    responseWriter.flush();
+  }
+
+  private static String getOneLineMessage(Throwable exception) {
+    String message = exception.getMessage();
+    if (message != null) {
+      int i = message.indexOf(ENTER);
+      if (i > -1) {
+        message = message.substring(0, i);
+      }
+    }
+    return message;
+  }
+}
diff --git a/knox-token-generation-ui/token-generation/app/token-generation.service.ts b/knox-token-generation-ui/token-generation/app/token-generation.service.ts
index 17f38eefe..b4b757cdc 100644
--- a/knox-token-generation-ui/token-generation/app/token-generation.service.ts
+++ b/knox-token-generation-ui/token-generation/app/token-generation.service.ts
@@ -118,15 +118,19 @@ export class TokenGenService {
     }
 
     private handleError(error: HttpErrorResponse): Promise<any> {
+	    let errorMsg = '';
+        if (error.error) {
+            errorMsg = error.error.RemoteException ? error.error.RemoteException.message : error.error;
+        }
         Swal.fire({
             icon: 'error',
             title: 'Oops!',
-            text: 'Something went wrong!\n' + (error.error ? error.error : error.statusText),
+            text: 'Something went wrong!\n' + (errorMsg ? errorMsg : error.statusText),
             confirmButtonColor: '#7cd1f9'
           });
         let requestErrorMessage = 'Response from ' + error.url + ' - ' + error.status + ': ' + error.statusText;
-        if (error.error) {
-            requestErrorMessage += ' (' + error.error + ')';
+        if (errorMsg) {
+            requestErrorMessage += ' (' + errorMsg + ')';
         }
         return Promise.reject(requestErrorMessage);
     }