You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@knox.apache.org by lm...@apache.org on 2013/09/10 22:39:36 UTC

git commit: KNOX-120 add acts based authorization provider. required changes to identity assertion as well.

Updated Branches:
  refs/heads/master 52eb9886b -> 660a724fe


KNOX-120 add acts based authorization provider. required changes to identity assertion as well.

Project: http://git-wip-us.apache.org/repos/asf/incubator-knox/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-knox/commit/660a724f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-knox/tree/660a724f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-knox/diff/660a724f

Branch: refs/heads/master
Commit: 660a724fe35ebb3822477f7bf7482ea18a00a225
Parents: 52eb988
Author: Larry McCay <lm...@hortonworks.com>
Authored: Tue Sep 10 16:32:38 2013 -0400
Committer: Larry McCay <lm...@hortonworks.com>
Committed: Tue Sep 10 16:32:38 2013 -0400

----------------------------------------------------------------------
 .../IdentityAsserterDeploymentContributor.java  |   3 +
 .../filter/IdentityAsserterFilter.java          |  13 +-
 gateway-provider-security-authz-acls/pom.xml    |  53 +++++
 .../impl/AclsAuthzDeploymentContributor.java    |  75 +++++++
 .../gateway/filter/AclsAuthorizationFilter.java | 222 +++++++++++++++++++
 .../filter/AclsAuthorizationMessages.java       |  66 ++++++
 ...gateway.deploy.ProviderDeploymentContributor |  18 ++
 .../filter/JWTAccessTokenAssertionFilter.java   |   2 +-
 .../jwt/filter/JWTAuthCodeAssertionFilter.java  |   2 +-
 gateway-release/pom.xml                         |   4 +
 gateway-server/pom.xml                          |   5 +
 .../gateway/deploy/DeploymentFactoryTest.java   |  25 ++-
 .../hbase/HbaseDeploymentContributor.java       |  14 +-
 .../gateway/hdfs/HdfsDeploymentContributor.java |  23 +-
 .../gateway/hive/HiveDeploymentContributor.java |  14 +-
 .../oozie/OozieDeploymentContributor.java       |  18 +-
 .../TempletonDeploymentContributor.java         |   1 +
 .../ServiceDeploymentContributorBase.java       |  24 ++
 .../AbstractIdentityAssertionFilter.java        | 122 +++++++++-
 .../hadoop/gateway/security/GroupPrincipal.java |   2 +-
 .../gateway/security/ImpersonatedPrincipal.java |  33 +++
 .../security/principal/PrincipalMapper.java     |  11 +-
 .../principal/PrincipalMappingException.java    |   4 +
 .../principal/SimplePrincipalMapper.java        |  88 +++++---
 .../security/principal/PrincipalMapperTest.java |  98 +++++---
 .../hadoop/gateway/GatewayBasicFuncTest.java    |  34 ++-
 pom.xml                                         |   6 +
 27 files changed, 830 insertions(+), 150 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-provider-identity-assertion-pseudo/src/main/java/org/apache/hadoop/gateway/identityasserter/filter/IdentityAsserterDeploymentContributor.java
----------------------------------------------------------------------
diff --git a/gateway-provider-identity-assertion-pseudo/src/main/java/org/apache/hadoop/gateway/identityasserter/filter/IdentityAsserterDeploymentContributor.java b/gateway-provider-identity-assertion-pseudo/src/main/java/org/apache/hadoop/gateway/identityasserter/filter/IdentityAsserterDeploymentContributor.java
index a7b1cda..bb5cd99 100644
--- a/gateway-provider-identity-assertion-pseudo/src/main/java/org/apache/hadoop/gateway/identityasserter/filter/IdentityAsserterDeploymentContributor.java
+++ b/gateway-provider-identity-assertion-pseudo/src/main/java/org/apache/hadoop/gateway/identityasserter/filter/IdentityAsserterDeploymentContributor.java
@@ -30,6 +30,7 @@ public class IdentityAsserterDeploymentContributor extends ProviderDeploymentCon
 
   private static final String FILTER_CLASSNAME = IdentityAsserterFilter.class.getName();
   private static final String PRINCIPAL_MAPPING_PARAM_NAME = "principal.mapping";
+  private static final String GROUP_PRINCIPAL_MAPPING_PARAM_NAME = "group.principal.mapping";
 
   @Override
   public String getRole() {
@@ -44,6 +45,7 @@ public class IdentityAsserterDeploymentContributor extends ProviderDeploymentCon
   @Override
   public void contributeProvider( DeploymentContext context, Provider provider ) {
     String mappings = provider.getParams().get(PRINCIPAL_MAPPING_PARAM_NAME);
+    String groupMappings = provider.getParams().get(GROUP_PRINCIPAL_MAPPING_PARAM_NAME);
 
 //    ServletType<WebAppDescriptor> servlet = findServlet( context, context.getTopology().getName() );
 //    servlet.createInitParam()
@@ -51,6 +53,7 @@ public class IdentityAsserterDeploymentContributor extends ProviderDeploymentCon
 //        .paramValue( mappings );
     
     context.getWebAppDescriptor().createContextParam().paramName(PRINCIPAL_MAPPING_PARAM_NAME).paramValue(mappings);
+    context.getWebAppDescriptor().createContextParam().paramName(GROUP_PRINCIPAL_MAPPING_PARAM_NAME).paramValue(groupMappings);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-provider-identity-assertion-pseudo/src/main/java/org/apache/hadoop/gateway/identityasserter/filter/IdentityAsserterFilter.java
----------------------------------------------------------------------
diff --git a/gateway-provider-identity-assertion-pseudo/src/main/java/org/apache/hadoop/gateway/identityasserter/filter/IdentityAsserterFilter.java b/gateway-provider-identity-assertion-pseudo/src/main/java/org/apache/hadoop/gateway/identityasserter/filter/IdentityAsserterFilter.java
index 3cafb82..b39fd90 100644
--- a/gateway-provider-identity-assertion-pseudo/src/main/java/org/apache/hadoop/gateway/identityasserter/filter/IdentityAsserterFilter.java
+++ b/gateway-provider-identity-assertion-pseudo/src/main/java/org/apache/hadoop/gateway/identityasserter/filter/IdentityAsserterFilter.java
@@ -43,14 +43,15 @@ public class IdentityAsserterFilter extends AbstractIdentityAssertionFilter {
     Subject subject = Subject.getSubject(AccessController.getContext());
 
     String principalName = getPrincipalName(subject);
-    principalName = mapper.mapPrincipal(principalName);
-//    System.out.println("+++++++++++++ Identity Assertion Filtering with Principal: " + principalName);
-
+    String mappedPrincipalName = mapper.mapUserPrincipal(principalName);
+    
+    // wrap the request so that the proper principal is returned
+    // from request methods
     IdentityAsserterHttpServletRequestWrapper wrapper =
         new IdentityAsserterHttpServletRequestWrapper(
         (HttpServletRequest)request, 
-        principalName);
-    chain.doFilter( wrapper, response );
-  }
+        mappedPrincipalName);
 
+    continueChainAsPrincipal(wrapper, response, chain, mappedPrincipalName);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-provider-security-authz-acls/pom.xml
----------------------------------------------------------------------
diff --git a/gateway-provider-security-authz-acls/pom.xml b/gateway-provider-security-authz-acls/pom.xml
new file mode 100644
index 0000000..e85b82a
--- /dev/null
+++ b/gateway-provider-security-authz-acls/pom.xml
@@ -0,0 +1,53 @@
+<!--
+   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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>gateway</artifactId>
+    <version>0.3.0-SNAPSHOT</version>
+  </parent>
+  <artifactId>gateway-provider-security-authz-acls</artifactId>
+  <dependencies>
+    <dependency>
+	    <groupId>${gateway-group}</groupId>
+	    <artifactId>gateway-spi</artifactId>
+	  </dependency>
+    <dependency>
+    	<groupId>commons-io</groupId>
+    	<artifactId>commons-io</artifactId>
+    </dependency>
+    
+    <dependency>
+        <groupId>junit</groupId>
+        <artifactId>junit</artifactId>
+        <scope>test</scope>
+    </dependency>
+
+    <dependency>
+        <groupId>org.easymock</groupId>
+        <artifactId>easymock</artifactId>
+        <scope>test</scope>
+    </dependency>
+
+    <dependency>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>gateway-test-utils</artifactId>
+        <scope>test</scope>
+    </dependency> 
+  </dependencies>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-provider-security-authz-acls/src/main/java/org/apache/hadoop/gateway/deploy/impl/AclsAuthzDeploymentContributor.java
----------------------------------------------------------------------
diff --git a/gateway-provider-security-authz-acls/src/main/java/org/apache/hadoop/gateway/deploy/impl/AclsAuthzDeploymentContributor.java b/gateway-provider-security-authz-acls/src/main/java/org/apache/hadoop/gateway/deploy/impl/AclsAuthzDeploymentContributor.java
new file mode 100644
index 0000000..2c150e3
--- /dev/null
+++ b/gateway-provider-security-authz-acls/src/main/java/org/apache/hadoop/gateway/deploy/impl/AclsAuthzDeploymentContributor.java
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.gateway.deploy.impl;
+
+import org.apache.hadoop.gateway.deploy.DeploymentContext;
+import org.apache.hadoop.gateway.deploy.ProviderDeploymentContributorBase;
+import org.apache.hadoop.gateway.descriptor.FilterParamDescriptor;
+import org.apache.hadoop.gateway.descriptor.ResourceDescriptor;
+import org.apache.hadoop.gateway.services.security.KeystoreService;
+import org.apache.hadoop.gateway.services.security.KeystoreServiceException;
+import org.apache.hadoop.gateway.topology.Provider;
+import org.apache.hadoop.gateway.topology.Service;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+public class AclsAuthzDeploymentContributor extends ProviderDeploymentContributorBase {
+
+  private static final String FILTER_CLASSNAME = "org.apache.hadoop.gateway.filter.AclsAuthorizationFilter";
+
+  @Override
+  public String getRole() {
+    return "authorization";
+  }
+
+  @Override
+  public String getName() {
+    return "AclsAuthz";
+  }
+
+  @Override
+  public void initializeContribution(DeploymentContext context) {
+    super.initializeContribution(context);
+  }
+
+  @Override
+  public void contributeProvider( DeploymentContext context, Provider provider ) {
+  }
+
+  @Override
+  public void contributeFilter( DeploymentContext context, Provider provider, Service service, 
+      ResourceDescriptor resource, List<FilterParamDescriptor> params ) {
+    if (params == null) {
+      params = new ArrayList<FilterParamDescriptor>();
+    }
+    // add resource role to params so that we can determine the acls to enforce at runtime
+    params.add( resource.createFilterParam().name( "resource.role" ).value(resource.role() ) );
+
+    // blindly add all the provider params as filter init params
+    // this will include any {resource.role}-ACLS parameters to be enforced - such as NAMENODE-ACLS
+    Map<String, String> providerParams = provider.getParams();
+    for(Entry<String, String> entry : providerParams.entrySet()) {
+      params.add( resource.createFilterParam().name( entry.getKey().toLowerCase() ).value( entry.getValue() ) );
+    }
+
+    resource.addFilter().name( getName() ).role( getRole() ).impl( FILTER_CLASSNAME ).params( params );
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-provider-security-authz-acls/src/main/java/org/apache/hadoop/gateway/filter/AclsAuthorizationFilter.java
----------------------------------------------------------------------
diff --git a/gateway-provider-security-authz-acls/src/main/java/org/apache/hadoop/gateway/filter/AclsAuthorizationFilter.java b/gateway-provider-security-authz-acls/src/main/java/org/apache/hadoop/gateway/filter/AclsAuthorizationFilter.java
new file mode 100644
index 0000000..f31cb0b
--- /dev/null
+++ b/gateway-provider-security-authz-acls/src/main/java/org/apache/hadoop/gateway/filter/AclsAuthorizationFilter.java
@@ -0,0 +1,222 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.gateway.filter;
+
+import javax.security.auth.Subject;
+import javax.servlet.Filter;
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.gateway.i18n.messages.MessagesFactory;
+import org.apache.hadoop.gateway.security.GroupPrincipal;
+import org.apache.hadoop.gateway.security.ImpersonatedPrincipal;
+import org.apache.hadoop.gateway.security.PrimaryPrincipal;
+
+import java.io.IOException;
+import java.security.AccessController;
+import java.security.Principal;
+import java.util.ArrayList;
+import java.util.Collections;
+
+public class AclsAuthorizationFilter implements Filter {
+  private static AclsAuthorizationMessages log = MessagesFactory.get( AclsAuthorizationMessages.class );
+
+  private String resourceRole = null;
+  private ArrayList<String> users;
+  private ArrayList<String> groups;
+  private ArrayList<String> ipaddr;
+  private String aclProcessingMode = null;
+  
+  @Override
+  public void init(FilterConfig filterConfig) throws ServletException {
+    resourceRole = getInitParameter(filterConfig, "resource.role");
+    log.initializingForResourceRole(resourceRole);
+    aclProcessingMode = getInitParameter(filterConfig, resourceRole + ".acl.processing.mode");
+    if (aclProcessingMode == null) {
+      aclProcessingMode = "OR";
+    }
+    log.aclProcessingMode(aclProcessingMode);
+    String acls = getInitParameter(filterConfig, resourceRole + ".acls");
+    parseAcls(acls);
+  }
+
+  private String getInitParameter(FilterConfig filterConfig, String paramName) {
+    return filterConfig.getInitParameter(paramName.toLowerCase());
+  }
+
+  private void parseAcls(String acls) {
+    if (acls != null) {
+      String[] parts = acls.split(";");
+      if (parts.length != 3 && parts.length > 0) {
+        log.invalidAclsFoundForResource(resourceRole);
+        // TODO: should probably throw an exception since this can leave
+        // us in an insecure state - either that or lock it down so that
+        // it isn't unprotected
+      }
+      else {
+        log.aclsFoundForResource(resourceRole);
+      }
+      users = new ArrayList<String>();
+      Collections.addAll(users, parts[0].split(","));
+  
+      groups = new ArrayList<String>();
+      Collections.addAll(groups, parts[1].split(","));
+  
+      ipaddr = new ArrayList<String>();
+      Collections.addAll(ipaddr, parts[2].split(","));
+    }
+    else {
+      log.noAclsFoundForResource(resourceRole);
+      users = new ArrayList<String>();
+      groups = new ArrayList<String>();
+      ipaddr = new ArrayList<String>();
+    }
+  }
+
+  public void destroy() {
+
+  }
+
+  public void doFilter(ServletRequest request, ServletResponse response,
+      FilterChain chain) throws IOException, ServletException {
+    boolean accessGranted = enforceAclAuthorizationPolicy(request, response, chain);
+    log.accessGranted(accessGranted);
+
+    if (accessGranted) {
+      chain.doFilter(request, response);
+    }
+    else {
+      sendUnauthorized((HttpServletResponse) response);
+    }
+  }
+
+  private boolean enforceAclAuthorizationPolicy(ServletRequest request,
+      ServletResponse response, FilterChain chain) {
+    HttpServletRequest req = (HttpServletRequest) request;
+    
+    // before enforcing acls check whether there are no acls defined 
+    // which would mean that there are no restrictions
+    if (users.size() == 0 && groups.size() == 0 && ipaddr.size() == 0) {
+      return true;
+    }
+
+    boolean userAccess = false;
+    boolean groupAccess = false;
+    boolean ipAddrAccess = false;
+    
+    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);
+    }
+    Object[] groups = subject.getPrincipals(GroupPrincipal.class).toArray();
+    if (groups.length > 0) {
+//      System.out.println("GroupPrincipal: " + ((Principal)groups[0]).getName());
+      groupAccess = checkGroupAcls(groups);
+      log.groupPrincipalHasAccess(groupAccess);
+    }
+    log.remoteIPAddress(req.getRemoteAddr());
+    ipAddrAccess = checkRemoteIpAcls(req.getRemoteAddr());
+    log.remoteIPAddressHasAccess(ipAddrAccess);
+    
+    if (aclProcessingMode.equals("OR")) {
+      return (userAccess || groupAccess || ipAddrAccess);
+    }
+    else if (aclProcessingMode.equals("AND")) {
+      return (userAccess && groupAccess && ipAddrAccess);
+    }
+    return false;
+  }
+
+  private boolean checkRemoteIpAcls(String remoteAddr) {
+    boolean allowed = false;
+    if (remoteAddr == null) {
+      return false;
+    }
+    if (ipaddr.contains("*")) {
+      allowed = true;
+    }
+    else {
+      if (ipaddr.contains(remoteAddr)) {
+        allowed = true;
+      }
+    }
+    return allowed;
+  }
+
+  private boolean checkUserAcls(Principal user) {
+    boolean allowed = false;
+    if (user == null) {
+      return false;
+    }
+    if (users.contains("*")) {
+      allowed = true;
+    }
+    else {
+      if (users.contains(user.getName())) {
+        allowed = true;
+      }
+    }
+    return allowed;
+  }
+
+  private boolean checkGroupAcls(Object[] userGroups) {
+    boolean allowed = false;
+    if (userGroups == null) {
+      return false;
+    }
+    if (groups.contains("*")) {
+      allowed = true;
+    }
+    else {
+      for (int i = 0; i < userGroups.length; i++) {
+        if (groups.contains(((Principal)userGroups[i]).getName())) {
+          allowed = true;
+          break;
+        }
+      }
+    }
+    return allowed;
+  }
+
+  private void sendUnauthorized(HttpServletResponse res) {
+    sendErrorCode(res, 401);
+  }
+
+  private void sendErrorCode(HttpServletResponse res, int code) {
+    try {
+      res.sendError(code);
+    } catch (IOException e) {
+      // TODO: log appropriately
+      e.printStackTrace();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-provider-security-authz-acls/src/main/java/org/apache/hadoop/gateway/filter/AclsAuthorizationMessages.java
----------------------------------------------------------------------
diff --git a/gateway-provider-security-authz-acls/src/main/java/org/apache/hadoop/gateway/filter/AclsAuthorizationMessages.java b/gateway-provider-security-authz-acls/src/main/java/org/apache/hadoop/gateway/filter/AclsAuthorizationMessages.java
new file mode 100644
index 0000000..071375d
--- /dev/null
+++ b/gateway-provider-security-authz-acls/src/main/java/org/apache/hadoop/gateway/filter/AclsAuthorizationMessages.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.gateway.filter;
+
+import org.apache.hadoop.gateway.i18n.messages.Message;
+import org.apache.hadoop.gateway.i18n.messages.MessageLevel;
+import org.apache.hadoop.gateway.i18n.messages.Messages;
+import org.apache.hadoop.gateway.i18n.messages.StackTrace;
+
+@Messages(logger="org.apache.hadoop.gateway")
+public interface AclsAuthorizationMessages {
+
+  @Message( level = MessageLevel.INFO, text = "Initializing AclsAuthz Provider for: {0}" )
+  void initializingForResourceRole(String resourceRole);
+
+  @Message( level = MessageLevel.DEBUG, text = "ACL Processing Mode is: {0}" )
+  void aclProcessingMode(String aclProcessingMode);
+
+  @Message( level = MessageLevel.WARN, text = "Invalid ACLs found for: {0}" )
+  void invalidAclsFoundForResource(String resourceRole);
+
+  @Message( level = MessageLevel.INFO, text = "ACLs found for: {0}" )
+  void aclsFoundForResource(String resourceRole);
+
+  @Message( level = MessageLevel.DEBUG, text = "No ACLs found for: {0}" )
+  void noAclsFoundForResource(String resourceRole);
+
+  @Message( level = MessageLevel.INFO, text = "Access Granted: {0}" )
+  void accessGranted(boolean accessGranted);
+
+  @Message( level = MessageLevel.DEBUG, text = "PrimaryPrincipal: {0}" )
+  void primaryPrincipal(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 = "GroupPrincipal has access: {0}" )
+  void groupPrincipalHasAccess(boolean groupAccess);
+
+  @Message( level = MessageLevel.DEBUG, text = "Remote IP Address: {0}" )
+  void remoteIPAddress(String remoteAddr);
+
+  @Message( level = MessageLevel.DEBUG, text = "Remote IP Address has access: {0}" )
+  void remoteIPAddressHasAccess(boolean remoteIpAccess);
+}

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-provider-security-authz-acls/src/main/resources/META-INF/services/org.apache.hadoop.gateway.deploy.ProviderDeploymentContributor
----------------------------------------------------------------------
diff --git a/gateway-provider-security-authz-acls/src/main/resources/META-INF/services/org.apache.hadoop.gateway.deploy.ProviderDeploymentContributor b/gateway-provider-security-authz-acls/src/main/resources/META-INF/services/org.apache.hadoop.gateway.deploy.ProviderDeploymentContributor
new file mode 100644
index 0000000..11e2f6f
--- /dev/null
+++ b/gateway-provider-security-authz-acls/src/main/resources/META-INF/services/org.apache.hadoop.gateway.deploy.ProviderDeploymentContributor
@@ -0,0 +1,18 @@
+##########################################################################
+# 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.
+##########################################################################
+org.apache.hadoop.gateway.deploy.impl.AclsAuthzDeploymentContributor

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-provider-security-jwt/src/main/java/org/apache/hadoop/gateway/provider/federation/jwt/filter/JWTAccessTokenAssertionFilter.java
----------------------------------------------------------------------
diff --git a/gateway-provider-security-jwt/src/main/java/org/apache/hadoop/gateway/provider/federation/jwt/filter/JWTAccessTokenAssertionFilter.java b/gateway-provider-security-jwt/src/main/java/org/apache/hadoop/gateway/provider/federation/jwt/filter/JWTAccessTokenAssertionFilter.java
index 6b8a41e..45d2b9d 100644
--- a/gateway-provider-security-jwt/src/main/java/org/apache/hadoop/gateway/provider/federation/jwt/filter/JWTAccessTokenAssertionFilter.java
+++ b/gateway-provider-security-jwt/src/main/java/org/apache/hadoop/gateway/provider/federation/jwt/filter/JWTAccessTokenAssertionFilter.java
@@ -86,7 +86,7 @@ public class JWTAccessTokenAssertionFilter extends AbstractIdentityAssertionFilt
 
       Subject subject = Subject.getSubject(AccessController.getContext());
       String principalName = getPrincipalName(subject);
-      principalName = mapper.mapPrincipal(principalName);
+      principalName = mapper.mapUserPrincipal(principalName);
       
       // calculate expiration timestamp: validity * 1000 + currentTimeInMillis
       long expires = System.currentTimeMillis() + validity * 1000;

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-provider-security-jwt/src/main/java/org/apache/hadoop/gateway/provider/federation/jwt/filter/JWTAuthCodeAssertionFilter.java
----------------------------------------------------------------------
diff --git a/gateway-provider-security-jwt/src/main/java/org/apache/hadoop/gateway/provider/federation/jwt/filter/JWTAuthCodeAssertionFilter.java b/gateway-provider-security-jwt/src/main/java/org/apache/hadoop/gateway/provider/federation/jwt/filter/JWTAuthCodeAssertionFilter.java
index ba691a5..8d649a5 100644
--- a/gateway-provider-security-jwt/src/main/java/org/apache/hadoop/gateway/provider/federation/jwt/filter/JWTAuthCodeAssertionFilter.java
+++ b/gateway-provider-security-jwt/src/main/java/org/apache/hadoop/gateway/provider/federation/jwt/filter/JWTAuthCodeAssertionFilter.java
@@ -62,7 +62,7 @@ public class JWTAuthCodeAssertionFilter extends AbstractIdentityAssertionFilter
 
       Subject subject = Subject.getSubject(AccessController.getContext());
       String principalName = getPrincipalName(subject);
-      principalName = mapper.mapPrincipal(principalName);
+      principalName = mapper.mapUserPrincipal(principalName);
       JWTToken authCode = authority.issueToken(subject, "RS256");
       
       // get the url for the token service

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-release/pom.xml
----------------------------------------------------------------------
diff --git a/gateway-release/pom.xml b/gateway-release/pom.xml
index 0a25213..2a5cdf1 100644
--- a/gateway-release/pom.xml
+++ b/gateway-release/pom.xml
@@ -159,6 +159,10 @@
         </dependency>
         <dependency>
             <groupId>${gateway-group}</groupId>
+            <artifactId>gateway-provider-security-authz-acls</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>${gateway-group}</groupId>
             <artifactId>gateway-provider-identity-assertion-pseudo</artifactId>
         </dependency>
 

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-server/pom.xml
----------------------------------------------------------------------
diff --git a/gateway-server/pom.xml b/gateway-server/pom.xml
index 877c9ed..c2c6d9d 100644
--- a/gateway-server/pom.xml
+++ b/gateway-server/pom.xml
@@ -219,6 +219,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.hadoop</groupId>
+            <artifactId>gateway-provider-security-authz-acls</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
             <artifactId>gateway-provider-identity-assertion-pseudo</artifactId>
             <scope>test</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-server/src/test/java/org/apache/hadoop/gateway/deploy/DeploymentFactoryTest.java
----------------------------------------------------------------------
diff --git a/gateway-server/src/test/java/org/apache/hadoop/gateway/deploy/DeploymentFactoryTest.java b/gateway-server/src/test/java/org/apache/hadoop/gateway/deploy/DeploymentFactoryTest.java
index 15b725a..1da6355 100644
--- a/gateway-server/src/test/java/org/apache/hadoop/gateway/deploy/DeploymentFactoryTest.java
+++ b/gateway-server/src/test/java/org/apache/hadoop/gateway/deploy/DeploymentFactoryTest.java
@@ -87,6 +87,11 @@ public class DeploymentFactoryTest {
     asserter.setName("Pseudo");
     asserter.setEnabled( true );
     topology.addProvider( asserter );
+    Provider authorizer = new Provider();
+    authorizer.setRole( "authorization" );
+    authorizer.setName("AclsAuthz");
+    authorizer.setEnabled( true );
+    topology.addProvider( authorizer );
 
     WebArchive war = DeploymentFactory.createDeployment( config, topology );
     //File dir = new File( System.getProperty( "user.dir" ) );
@@ -120,9 +125,13 @@ public class DeploymentFactoryTest {
     assertThat( gateway, hasXPath( "/gateway/resource[1]/filter[5]/role", equalTo( "identity-assertion" ) ) );
     assertThat( gateway, hasXPath( "/gateway/resource[1]/filter[5]/class", equalTo( "org.apache.hadoop.gateway.identityasserter.filter.IdentityAsserterFilter" ) ) );
 
-    assertThat( gateway, hasXPath( "/gateway/resource[1]/filter[6]/role", equalTo( "dispatch" ) ) );
-    assertThat( gateway, hasXPath( "/gateway/resource[1]/filter[6]/name", equalTo( "http-client" ) ) );
-    assertThat( gateway, hasXPath( "/gateway/resource[1]/filter[6]/class", equalTo( "org.apache.hadoop.gateway.dispatch.HttpClientDispatch" ) ) );
+    assertThat( gateway, hasXPath( "/gateway/resource[1]/filter[6]/role", equalTo( "authorization" ) ) );
+    assertThat( gateway, hasXPath( "/gateway/resource[1]/filter[6]/name", equalTo( "AclsAuthz" ) ) );
+    assertThat( gateway, hasXPath( "/gateway/resource[1]/filter[6]/class", equalTo( "org.apache.hadoop.gateway.filter.AclsAuthorizationFilter" ) ) );
+
+    assertThat( gateway, hasXPath( "/gateway/resource[1]/filter[7]/role", equalTo( "dispatch" ) ) );
+    assertThat( gateway, hasXPath( "/gateway/resource[1]/filter[7]/name", equalTo( "http-client" ) ) );
+    assertThat( gateway, hasXPath( "/gateway/resource[1]/filter[7]/class", equalTo( "org.apache.hadoop.gateway.dispatch.HttpClientDispatch" ) ) );
 
     assertThat( gateway, hasXPath( "/gateway/resource[2]/pattern", equalTo( "/namenode/api/v1/**?**" ) ) );
     //assertThat( gateway, hasXPath( "/gateway/resource[2]/target", equalTo( "http://localhost:50070/webhdfs/v1/{path=**}?{**}" ) ) );
@@ -142,9 +151,13 @@ public class DeploymentFactoryTest {
     assertThat( gateway, hasXPath( "/gateway/resource[2]/filter[5]/role", equalTo( "identity-assertion" ) ) );
     assertThat( gateway, hasXPath( "/gateway/resource[2]/filter[5]/class", equalTo( "org.apache.hadoop.gateway.identityasserter.filter.IdentityAsserterFilter" ) ) );
 
-    assertThat( gateway, hasXPath( "/gateway/resource[2]/filter[6]/role", equalTo( "dispatch" ) ) );
-    assertThat( gateway, hasXPath( "/gateway/resource[2]/filter[6]/name", equalTo( "http-client" ) ) );
-    assertThat( gateway, hasXPath( "/gateway/resource[2]/filter[6]/class", equalTo( "org.apache.hadoop.gateway.dispatch.HttpClientDispatch" ) ) );
+    assertThat( gateway, hasXPath( "/gateway/resource[1]/filter[6]/role", equalTo( "authorization" ) ) );
+    assertThat( gateway, hasXPath( "/gateway/resource[1]/filter[6]/name", equalTo( "AclsAuthz" ) ) );
+    assertThat( gateway, hasXPath( "/gateway/resource[1]/filter[6]/class", equalTo( "org.apache.hadoop.gateway.filter.AclsAuthorizationFilter" ) ) );
+
+    assertThat( gateway, hasXPath( "/gateway/resource[2]/filter[7]/role", equalTo( "dispatch" ) ) );
+    assertThat( gateway, hasXPath( "/gateway/resource[2]/filter[7]/name", equalTo( "http-client" ) ) );
+    assertThat( gateway, hasXPath( "/gateway/resource[2]/filter[7]/class", equalTo( "org.apache.hadoop.gateway.dispatch.HttpClientDispatch" ) ) );
   }
 
   private Document parse( InputStream stream ) throws IOException, SAXException, ParserConfigurationException {

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-service-hbase/src/main/java/org/apache/hadoop/gateway/hbase/HbaseDeploymentContributor.java
----------------------------------------------------------------------
diff --git a/gateway-service-hbase/src/main/java/org/apache/hadoop/gateway/hbase/HbaseDeploymentContributor.java b/gateway-service-hbase/src/main/java/org/apache/hadoop/gateway/hbase/HbaseDeploymentContributor.java
index 1e3f2be..c467fdf 100644
--- a/gateway-service-hbase/src/main/java/org/apache/hadoop/gateway/hbase/HbaseDeploymentContributor.java
+++ b/gateway-service-hbase/src/main/java/org/apache/hadoop/gateway/hbase/HbaseDeploymentContributor.java
@@ -99,18 +99,10 @@ public class HbaseDeploymentContributor extends ServiceDeploymentContributorBase
     addAuthenticationFilter( context, service, fileResource );
     addRewriteFilter( context, service, fileResource );
     addIdentityAssertionFilter( context, service, fileResource );
+    addAuthorizationFilter(context, service, fileResource);
     addDispatchFilter( context, service, fileResource );
   }
 
-  private void addAuthenticationFilter( DeploymentContext context, Service service, ResourceDescriptor resource ) {
-    if (topologyContainsProviderType( context, "authentication" )) {
-      context.contributeFilter( service, resource, "authentication", null, null );
-    }
-    if ( topologyContainsProviderType( context, "federation" ) ) {
-      context.contributeFilter( service, resource, "federation", null, null );
-    }
-  }
-
   private void addRewriteFilter(
       DeploymentContext context, Service service, ResourceDescriptor resource ) throws URISyntaxException {
     List<FilterParamDescriptor> params = new ArrayList<FilterParamDescriptor>();
@@ -118,10 +110,6 @@ public class HbaseDeploymentContributor extends ServiceDeploymentContributorBase
     context.contributeFilter( service, resource, "rewrite", null, params );
   }
 
-  private void addIdentityAssertionFilter(DeploymentContext context, Service service, ResourceDescriptor resource) {
-    context.contributeFilter( service, resource, "identity-assertion", null, null );
-  }
-
   private void addDispatchFilter(
       DeploymentContext context, Service service, ResourceDescriptor resource ) {
     context.contributeFilter( service, resource, "dispatch", null, null );

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-service-hdfs/src/main/java/org/apache/hadoop/gateway/hdfs/HdfsDeploymentContributor.java
----------------------------------------------------------------------
diff --git a/gateway-service-hdfs/src/main/java/org/apache/hadoop/gateway/hdfs/HdfsDeploymentContributor.java b/gateway-service-hdfs/src/main/java/org/apache/hadoop/gateway/hdfs/HdfsDeploymentContributor.java
index 29a257e..76f5dfb 100644
--- a/gateway-service-hdfs/src/main/java/org/apache/hadoop/gateway/hdfs/HdfsDeploymentContributor.java
+++ b/gateway-service-hdfs/src/main/java/org/apache/hadoop/gateway/hdfs/HdfsDeploymentContributor.java
@@ -107,6 +107,7 @@ public class HdfsDeploymentContributor extends ServiceDeploymentContributorBase
     addAuthenticationFilter( context, service, rootResource );
     addRewriteFilter( context, service, rootResource );
     addIdentityAssertionFilter( context, service, rootResource );
+    addAuthorizationFilter( context, service, rootResource );
     addDispatchFilter( context, service, rootResource, "dispatch", null );
 
     ResourceDescriptor fileResource = context.getGatewayDescriptor().addResource();
@@ -115,6 +116,7 @@ public class HdfsDeploymentContributor extends ServiceDeploymentContributorBase
     addAuthenticationFilter( context, service, fileResource );
     addRewriteFilter( context, service, fileResource );
     addIdentityAssertionFilter( context, service, fileResource );
+    addAuthorizationFilter( context, service, fileResource );
     addDispatchFilter( context, service, fileResource, "dispatch", null );
   }
 
@@ -123,29 +125,12 @@ public class HdfsDeploymentContributor extends ServiceDeploymentContributorBase
     fileResource.role( service.getRole() );
     fileResource.pattern( DATANODE_EXTERNAL_PATH + "/**?**" );
     addAuthenticationFilter( context, service, fileResource );
-    addRewriteFilter( context, service, fileResource );
     addIdentityAssertionFilter( context, service, fileResource );
+    addAuthorizationFilter( context, service, fileResource );
+    addRewriteFilter( context, service, fileResource );
     addDispatchFilter( context, service, fileResource, "dispatch", null );
   }
 
-  private void addAuthenticationFilter( DeploymentContext context, Service service, ResourceDescriptor resource ) {
-    if (topologyContainsProviderType(context, "authentication")) {
-      context.contributeFilter( service, resource, "authentication", null, null );
-    }
-    if (topologyContainsProviderType(context, "federation")) {
-      context.contributeFilter( service, resource, "federation", null, null );
-    }
-  }
-
-  private void addIdentityAssertionFilter(DeploymentContext context, Service service, ResourceDescriptor resource) {
-    context.contributeFilter( service, resource, "identity-assertion", null, null );
-  }
-
-  private void addDispatchFilter(
-    DeploymentContext context, Service service, ResourceDescriptor resource, String role, String name ) {
-    context.contributeFilter( service, resource, role, name, null );
-  }
-
   private void addRewriteFilter(
       DeploymentContext context, Service service, ResourceDescriptor resource ) throws URISyntaxException {
     List<FilterParamDescriptor> params = new ArrayList<FilterParamDescriptor>();

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-service-hive/src/main/java/org/apache/hadoop/gateway/hive/HiveDeploymentContributor.java
----------------------------------------------------------------------
diff --git a/gateway-service-hive/src/main/java/org/apache/hadoop/gateway/hive/HiveDeploymentContributor.java b/gateway-service-hive/src/main/java/org/apache/hadoop/gateway/hive/HiveDeploymentContributor.java
index 59d247c..97890ec 100644
--- a/gateway-service-hive/src/main/java/org/apache/hadoop/gateway/hive/HiveDeploymentContributor.java
+++ b/gateway-service-hive/src/main/java/org/apache/hadoop/gateway/hive/HiveDeploymentContributor.java
@@ -68,27 +68,15 @@ public class HiveDeploymentContributor extends ServiceDeploymentContributorBase
     addAuthenticationFilter( context, service, rootResource );
     addRewriteFilter( context, service, rootResource );
     addIdentityAssertionFilter( context, service, rootResource );
+    addAuthorizationFilter(context, service, rootResource);
     addDispatchFilter( context, service, rootResource );
   }
 
-  private void addAuthenticationFilter( DeploymentContext context, Service service, ResourceDescriptor resource ) {
-    if( topologyContainsProviderType( context, "authentication" ) ) {
-      context.contributeFilter( service, resource, "authentication", null, null );
-    }
-    if( topologyContainsProviderType( context, "federation" ) ) {
-      context.contributeFilter( service, resource, "federation", null, null );
-    }
-  }
-
   private void addRewriteFilter(
       DeploymentContext context, Service service, ResourceDescriptor resource ) throws URISyntaxException {
     context.contributeFilter( service, resource, "rewrite", null, null );
   }
 
-  private void addIdentityAssertionFilter( DeploymentContext context, Service service, ResourceDescriptor resource ) {
-    context.contributeFilter( service, resource, "identity-assertion", null, null );
-  }
-
   private void addDispatchFilter( DeploymentContext context, Service service, ResourceDescriptor resource ) {
     context.contributeFilter( service, resource, "dispatch", null, null );
   }

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-service-oozie/src/main/java/org/apache/hadoop/gateway/oozie/OozieDeploymentContributor.java
----------------------------------------------------------------------
diff --git a/gateway-service-oozie/src/main/java/org/apache/hadoop/gateway/oozie/OozieDeploymentContributor.java b/gateway-service-oozie/src/main/java/org/apache/hadoop/gateway/oozie/OozieDeploymentContributor.java
index 8ac1f31..6a16f62 100644
--- a/gateway-service-oozie/src/main/java/org/apache/hadoop/gateway/oozie/OozieDeploymentContributor.java
+++ b/gateway-service-oozie/src/main/java/org/apache/hadoop/gateway/oozie/OozieDeploymentContributor.java
@@ -72,6 +72,7 @@ public class OozieDeploymentContributor extends ServiceDeploymentContributorBase
     addAuthenticationFilter( context, service, rootResource );
     addRewriteFilter( context, service, rootResource );
     addIdentityAssertionFilter( context, service, rootResource );
+    addAuthorizationFilter(context, service, rootResource);
     addDispatchFilter( context, service, rootResource );
 
     ResourceDescriptor apiResource = context.getGatewayDescriptor().addResource();
@@ -80,29 +81,16 @@ public class OozieDeploymentContributor extends ServiceDeploymentContributorBase
     addAuthenticationFilter( context, service, apiResource );
     addRewriteFilter( context, service, apiResource );
     addIdentityAssertionFilter( context, service, apiResource );
+    addAuthorizationFilter(context, service, apiResource);
     addDispatchFilter( context, service, apiResource );
   }
 
-  private void addAuthenticationFilter( DeploymentContext context, Service service, ResourceDescriptor resource ) {
-    if (topologyContainsProviderType(context, "authentication")) {
-      context.contributeFilter( service, resource, "authentication", null, null );
-    }
-    if (topologyContainsProviderType(context, "federation")) {
-      context.contributeFilter( service, resource, "federation", null, null );
-    }
-  }
-
   private void addRewriteFilter(
       DeploymentContext context, Service service, ResourceDescriptor resource ) throws URISyntaxException {
     context.contributeFilter( service, resource, "rewrite", null, null );
   }
 
-  private void addIdentityAssertionFilter(DeploymentContext context, Service service, ResourceDescriptor resource) {
-    context.contributeFilter( service, resource, "identity-assertion", null, null );
-  }
-
-  private void addDispatchFilter(
-      DeploymentContext context, Service service, ResourceDescriptor resource ) {
+  private void addDispatchFilter(DeploymentContext context, Service service, ResourceDescriptor resource ) {
     context.contributeFilter( service, resource, "dispatch", null, null );
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-service-templeton/src/main/java/org/apache/hadoop/gateway/templeton/TempletonDeploymentContributor.java
----------------------------------------------------------------------
diff --git a/gateway-service-templeton/src/main/java/org/apache/hadoop/gateway/templeton/TempletonDeploymentContributor.java b/gateway-service-templeton/src/main/java/org/apache/hadoop/gateway/templeton/TempletonDeploymentContributor.java
index fcaf5cc..88ed2cd 100644
--- a/gateway-service-templeton/src/main/java/org/apache/hadoop/gateway/templeton/TempletonDeploymentContributor.java
+++ b/gateway-service-templeton/src/main/java/org/apache/hadoop/gateway/templeton/TempletonDeploymentContributor.java
@@ -64,6 +64,7 @@ public class TempletonDeploymentContributor extends ServiceDeploymentContributor
     }
     context.contributeFilter( service, resource, "rewrite", null, null );
     context.contributeFilter( service, resource, "identity-assertion", null, null );
+    addAuthorizationFilter(context, service, resource);
     context.contributeFilter( service, resource, "dispatch", null, null );
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-spi/src/main/java/org/apache/hadoop/gateway/deploy/ServiceDeploymentContributorBase.java
----------------------------------------------------------------------
diff --git a/gateway-spi/src/main/java/org/apache/hadoop/gateway/deploy/ServiceDeploymentContributorBase.java b/gateway-spi/src/main/java/org/apache/hadoop/gateway/deploy/ServiceDeploymentContributorBase.java
index fbd9865..4edfde8 100644
--- a/gateway-spi/src/main/java/org/apache/hadoop/gateway/deploy/ServiceDeploymentContributorBase.java
+++ b/gateway-spi/src/main/java/org/apache/hadoop/gateway/deploy/ServiceDeploymentContributorBase.java
@@ -19,7 +19,9 @@ package org.apache.hadoop.gateway.deploy;
 
 import java.util.Collection;
 
+import org.apache.hadoop.gateway.descriptor.ResourceDescriptor;
 import org.apache.hadoop.gateway.topology.Provider;
+import org.apache.hadoop.gateway.topology.Service;
 
 public abstract class ServiceDeploymentContributorBase extends DeploymentContributorBase implements ServiceDeploymentContributor {
 
@@ -47,5 +49,27 @@ public abstract class ServiceDeploymentContributorBase extends DeploymentContrib
     }
     return p;
   }
+  
+  protected void addAuthenticationFilter( DeploymentContext context, Service service, ResourceDescriptor resource ) {
+    if (topologyContainsProviderType(context, "authentication")) {
+      context.contributeFilter( service, resource, "authentication", null, null );
+    }
+    if (topologyContainsProviderType(context, "federation")) {
+      context.contributeFilter( service, resource, "federation", null, null );
+    }
+  }
+
+  protected void addIdentityAssertionFilter(DeploymentContext context, Service service, ResourceDescriptor resource) {
+    context.contributeFilter( service, resource, "identity-assertion", null, null );
+  }
+
+  protected void addAuthorizationFilter(DeploymentContext context, Service service, ResourceDescriptor resource) {
+    if (topologyContainsProviderType(context, "authorization")) {
+      context.contributeFilter( service, resource, "authorization", null, null );
+    }
+  }
 
+  protected void addDispatchFilter(DeploymentContext context, Service service, ResourceDescriptor resource, String role, String name ) {
+    context.contributeFilter( service, resource, role, name, null );
+  }  
 }

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-spi/src/main/java/org/apache/hadoop/gateway/filter/security/AbstractIdentityAssertionFilter.java
----------------------------------------------------------------------
diff --git a/gateway-spi/src/main/java/org/apache/hadoop/gateway/filter/security/AbstractIdentityAssertionFilter.java b/gateway-spi/src/main/java/org/apache/hadoop/gateway/filter/security/AbstractIdentityAssertionFilter.java
index 2090b9d..6246fc4 100644
--- a/gateway-spi/src/main/java/org/apache/hadoop/gateway/filter/security/AbstractIdentityAssertionFilter.java
+++ b/gateway-spi/src/main/java/org/apache/hadoop/gateway/filter/security/AbstractIdentityAssertionFilter.java
@@ -19,12 +19,27 @@ package org.apache.hadoop.gateway.filter.security;
 
 import org.apache.hadoop.gateway.i18n.GatewaySpiMessages;
 import org.apache.hadoop.gateway.i18n.messages.MessagesFactory;
+import org.apache.hadoop.gateway.security.GroupPrincipal;
+import org.apache.hadoop.gateway.security.ImpersonatedPrincipal;
+import org.apache.hadoop.gateway.security.PrimaryPrincipal;
 import org.apache.hadoop.gateway.security.principal.PrincipalMapper;
 import org.apache.hadoop.gateway.security.principal.PrincipalMappingException;
 import org.apache.hadoop.gateway.security.principal.SimplePrincipalMapper;
 
+import javax.security.auth.Subject;
+
+import java.io.IOException;
+import java.security.AccessController;
+import java.security.Principal;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+
 import javax.servlet.Filter;
+import javax.servlet.FilterChain;
 import javax.servlet.FilterConfig;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
 import javax.servlet.ServletException;
 
 public abstract class AbstractIdentityAssertionFilter extends AbstractIdentityAssertionBase implements Filter {
@@ -38,9 +53,10 @@ public abstract class AbstractIdentityAssertionFilter extends AbstractIdentityAs
 
   protected void loadPrincipalMappings(FilterConfig filterConfig) {
     String principalMapping = filterConfig.getServletContext().getInitParameter("principal.mapping");
-    if (principalMapping != null && !principalMapping.isEmpty()) {
+    String groupMapping = filterConfig.getServletContext().getInitParameter("group.principal.mapping");
+    if (principalMapping != null && !principalMapping.isEmpty() || groupMapping != null && !groupMapping.isEmpty()) {
       try {
-        mapper.loadMappingTable(principalMapping);
+        mapper.loadMappingTable(principalMapping, groupMapping);
       }
       catch (PrincipalMappingException pme) {
         LOG.failedToLoadPrincipalMappingTable( pme );
@@ -58,4 +74,106 @@ public abstract class AbstractIdentityAssertionFilter extends AbstractIdentityAs
     
   }
 
+  /**
+   * Recreate the current Subject based upon the provided mappedPrincipal
+   * and look for the groups that should be associated with the new Subject.
+   * Upon finding groups mapped to the principal - add them to the new Subject.
+   * @param mappedPrincipalName
+   * @throws ServletException 
+   * @throws IOException 
+   */
+  protected void continueChainAsPrincipal(final ServletRequest request, final ServletResponse response, 
+      final FilterChain chain, String mappedPrincipalName) throws IOException, ServletException {
+    Subject subject = null;
+    Principal impersonationPrincipal = null;
+    Principal primaryPrincipal = null;
+    
+    // get the current subject and determine whether we need another doAs with 
+    // an impersonatedPrincipal and/or mapped group principals
+    boolean impersonationNeeded = false;
+    boolean groupsMapped = false;
+    Subject currentSubject = Subject.getSubject(AccessController.getContext());
+    primaryPrincipal = (PrimaryPrincipal) currentSubject.getPrincipals(PrimaryPrincipal.class).toArray()[0];
+    if (primaryPrincipal != null) {
+      if (!primaryPrincipal.getName().equals(mappedPrincipalName)) {
+        impersonationNeeded = true;
+      }
+    }
+    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());
+    }
+    groupsMapped = areGroupsMappedForPrincipal(mappedPrincipalName);
+    
+    if (impersonationNeeded || groupsMapped) {
+      // gonna need a new subject and doAs
+      subject = new Subject();
+      subject.getPrincipals().add(primaryPrincipal);
+      if (impersonationNeeded) {
+        impersonationPrincipal = new ImpersonatedPrincipal(mappedPrincipalName);
+        subject.getPrincipals().add(impersonationPrincipal);
+      }
+      if (groupsMapped) {
+        addMappedGroupsToSubject(mappedPrincipalName, subject);
+        addMappedGroupsToSubject("*", subject);
+      }
+      doAs(request, response, chain, subject);
+    }
+    else {
+      doFilterInternal(request, response, chain);
+    }
+  }
+
+  private void doAs(final ServletRequest request,
+      final ServletResponse response, final FilterChain chain, Subject subject)
+      throws IOException, ServletException {
+    try {
+      Subject.doAs(
+          subject,
+          new PrivilegedExceptionAction<Object>() {
+            public Object run() throws Exception {
+              doFilterInternal(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);
+      }
+    }
+  }
+
+  private void addMappedGroupsToSubject(String mappedPrincipalName, Subject subject) {
+    String[] groups = mapper.mapGroupPrincipal(mappedPrincipalName);
+    if (groups != null) {
+      for (int i = 0; i < groups.length; i++) {
+        subject.getPrincipals().add(new GroupPrincipal(groups[i]));
+      }
+    }
+  }
+  
+  private boolean areGroupsMappedForPrincipal(String principalName) {
+    boolean mapped = false;
+    mapped = (mapper.mapGroupPrincipal(principalName) != null ? true : false);
+    if (!mapped) {
+      mapped = (mapper.mapGroupPrincipal("*") != null ? true : false);
+    }
+    return mapped;
+  }
+
+  private void doFilterInternal(ServletRequest request, ServletResponse response, FilterChain chain) throws IOException, ServletException {
+    chain.doFilter(request, response);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/GroupPrincipal.java
----------------------------------------------------------------------
diff --git a/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/GroupPrincipal.java b/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/GroupPrincipal.java
index e822269..7b4bb4e 100644
--- a/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/GroupPrincipal.java
+++ b/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/GroupPrincipal.java
@@ -23,7 +23,7 @@ public class GroupPrincipal implements Principal {
   private String name = null;
 
   public GroupPrincipal(String name) {
-    
+    this.name = name;
   }
   
   public String getName() {

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/ImpersonatedPrincipal.java
----------------------------------------------------------------------
diff --git a/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/ImpersonatedPrincipal.java b/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/ImpersonatedPrincipal.java
new file mode 100644
index 0000000..52c3371
--- /dev/null
+++ b/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/ImpersonatedPrincipal.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.gateway.security;
+
+import java.security.Principal;
+
+public class ImpersonatedPrincipal implements Principal {
+  private String name = null;
+  
+  public ImpersonatedPrincipal(String name) {
+    this.name = name;
+  }
+  
+  @Override
+  public String getName() {
+    return name;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/principal/PrincipalMapper.java
----------------------------------------------------------------------
diff --git a/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/principal/PrincipalMapper.java b/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/principal/PrincipalMapper.java
index f862a94..5685886 100644
--- a/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/principal/PrincipalMapper.java
+++ b/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/principal/PrincipalMapper.java
@@ -25,7 +25,7 @@ public interface PrincipalMapper {
    * actual[,another-actual]=mapped;...
    * @param principalMapping
    */
-  public abstract void loadMappingTable(String principalMapping)
+  public abstract void loadMappingTable(String principalMapping, String groupMapping)
       throws PrincipalMappingException;
 
   /**
@@ -35,6 +35,13 @@ public interface PrincipalMapper {
    * @param principalName
    * @return principal name to be used in the assertion
    */
-  public abstract String mapPrincipal(String principalName);
+  public abstract String mapUserPrincipal(String principalName);
 
+  /**
+   * Acquire array of group principal names from the mapping table
+   * as appropriate. Otherwise, return null.
+   * @param principalName
+   * @return group principal names to be used in the assertion
+   */
+  public abstract String[] mapGroupPrincipal(String principalName);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/principal/PrincipalMappingException.java
----------------------------------------------------------------------
diff --git a/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/principal/PrincipalMappingException.java b/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/principal/PrincipalMappingException.java
index 587d0d1..4e0b249 100644
--- a/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/principal/PrincipalMappingException.java
+++ b/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/principal/PrincipalMappingException.java
@@ -23,4 +23,8 @@ public class PrincipalMappingException extends Exception {
     super(message);
   }
 
+  public PrincipalMappingException(String message, Exception e) {
+    super(message, e);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/principal/SimplePrincipalMapper.java
----------------------------------------------------------------------
diff --git a/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/principal/SimplePrincipalMapper.java b/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/principal/SimplePrincipalMapper.java
index b3770e6..7947ba3 100644
--- a/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/principal/SimplePrincipalMapper.java
+++ b/gateway-spi/src/main/java/org/apache/hadoop/gateway/security/principal/SimplePrincipalMapper.java
@@ -21,7 +21,8 @@ import java.util.HashMap;
 import java.util.StringTokenizer;
 
 public class SimplePrincipalMapper implements PrincipalMapper {
-  public HashMap<String, String> table = new HashMap<String, String>();
+  public HashMap<String, String[]> principalMappings = null;
+  public HashMap<String, String[]> groupMappings = null;
 
   public SimplePrincipalMapper() {
   }
@@ -30,30 +31,42 @@ public class SimplePrincipalMapper implements PrincipalMapper {
    * @see org.apache.hadoop.gateway.filter.PrincipalMapper#loadMappingTable(java.lang.String)
    */
   @Override
-  public void loadMappingTable(String principalMapping) throws PrincipalMappingException {
+  public void loadMappingTable(String principalMapping, String groupMapping) throws PrincipalMappingException {
 //    System.out.println("+++++++++++++ Loading the Mapping Table");
     if (principalMapping != null) {
-      try {
-        StringTokenizer t = new StringTokenizer(principalMapping, ";");
-        do {
-          String mapping = t.nextToken();
-  //        System.out.println("+++++++++++++ Mapping: " + mapping);
-          String principals = mapping.substring(0, mapping.indexOf('='));
-  //        System.out.println("+++++++++++++ Principals: " + principals);
-          String value = mapping.substring(mapping.indexOf('=')+1);
-          String[] p = principals.split(",");
-          for(int i = 0; i < p.length; i++) {
-            table.put(p[i], value);
-  //          System.out.println("+++++++++++++ Mapping into Table: " + p[i] + "->" + value);
-          }
-        } while(t.hasMoreTokens());
-      }
-      catch (Exception e) {
-        // do not leave table in an unknown state - clear it instead
-        // no principal mapping will occur
-        table.clear();
-        throw new PrincipalMappingException("Unable to load mappings from provided string: " + principalMapping + " - no principal mapping will be provided.");
-      }
+      principalMappings = parseMapping(principalMapping);
+      groupMappings = parseMapping(groupMapping);
+    }
+  }
+
+  private HashMap<String, String[]> parseMapping(String mappings)
+      throws PrincipalMappingException {
+    if (mappings == null) {
+      return null;
+    }
+    HashMap<String, String[]> table = new HashMap<String, String[]>();
+    try {
+      StringTokenizer t = new StringTokenizer(mappings, ";");
+      do {
+        String mapping = t.nextToken();
+ //        System.out.println("+++++++++++++ Mapping: " + mapping);
+        String principals = mapping.substring(0, mapping.indexOf('='));
+ //        System.out.println("+++++++++++++ Principals: " + principals);
+        String value = mapping.substring(mapping.indexOf('=')+1);
+        String[] v = value.split(",");
+        String[] p = principals.split(",");
+        for(int i = 0; i < p.length; i++) {
+          table.put(p[i], v);
+ //          System.out.println("+++++++++++++ Mapping into Table: " + p[i] + "->" + value);
+        }
+      } while(t.hasMoreTokens());
+      return table;
+    }
+    catch (Exception e) {
+      // do not leave table in an unknown state - clear it instead
+      // no principal mapping will occur
+      table.clear();
+      throw new PrincipalMappingException("Unable to load mappings from provided string: " + mappings + " - no principal mapping will be provided.", e);
     }
   }
   
@@ -61,14 +74,29 @@ public class SimplePrincipalMapper implements PrincipalMapper {
    * @see org.apache.hadoop.gateway.filter.PrincipalMapper#mapPrincipal(java.lang.String)
    */
   @Override
-  public String mapPrincipal(String principalName) {
-    String p = null;
-    
-    p = table.get(principalName);
+  public String mapUserPrincipal(String principalName) {
+    String[] p = null;
+    if (principalMappings != null) {
+      p = principalMappings.get(principalName);
+    }
     if (p == null) {
-      p = principalName;
+      return principalName;
+    }
+    
+    return p[0];
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.gateway.filter.PrincipalMapper#mapPrincipal(java.lang.String)
+   */
+  @Override
+  public String[] mapGroupPrincipal(String principalName) {
+    String[] groups = null;
+    
+    if (groupMappings != null) {
+      groups = groupMappings.get(principalName);
     }
     
-    return p;
+    return groups;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-spi/src/test/java/org/apache/hadoop/gateway/security/principal/PrincipalMapperTest.java
----------------------------------------------------------------------
diff --git a/gateway-spi/src/test/java/org/apache/hadoop/gateway/security/principal/PrincipalMapperTest.java b/gateway-spi/src/test/java/org/apache/hadoop/gateway/security/principal/PrincipalMapperTest.java
index 2a0edc0..e50f5b2 100644
--- a/gateway-spi/src/test/java/org/apache/hadoop/gateway/security/principal/PrincipalMapperTest.java
+++ b/gateway-spi/src/test/java/org/apache/hadoop/gateway/security/principal/PrincipalMapperTest.java
@@ -39,107 +39,135 @@ public class PrincipalMapperTest {
   }
   
   @Test
+  public void testNonNullSimplePrincipalMappingWithGroups() {
+    String principalMapping = "lmccay,kminder=hdfs;newuser=mapred";
+    String groupMapping = "hdfs=group1;mapred=mrgroup,mrducks";
+    try {
+      mapper.loadMappingTable(principalMapping, groupMapping);
+    }
+    catch (PrincipalMappingException pme) {
+      pme.printStackTrace();
+      fail();
+    }
+    
+    assertTrue(mapper.mapUserPrincipal("lmccay").equals("hdfs"));
+    assertTrue(mapper.mapGroupPrincipal("hdfs")[0].equals("group1"));
+
+    assertTrue(mapper.mapUserPrincipal("kminder").equals("hdfs"));
+    
+    assertTrue(mapper.mapUserPrincipal("newuser").equals("mapred"));
+    assertTrue(mapper.mapGroupPrincipal("mapred")[0].equals("mrgroup"));
+    assertTrue(mapper.mapGroupPrincipal("mapred")[1].equals("mrducks"));
+
+    assertTrue(mapper.mapUserPrincipal("hdfs").equals("hdfs"));
+    assertTrue(mapper.mapUserPrincipal("mapred").equals("mapred"));
+
+    assertTrue(mapper.mapUserPrincipal("stink").equals("stink"));
+  }
+
+  @Test
   public void testNonNullSimplePrincipalMapping() {
     String principalMapping = "lmccay,kminder=hdfs;newuser=mapred";
     try {
-      mapper.loadMappingTable(principalMapping);
+      mapper.loadMappingTable(principalMapping, null);
     }
     catch (PrincipalMappingException pme) {
+      pme.printStackTrace();
       fail();
     }
     
-    assertTrue(mapper.mapPrincipal("lmccay").equals("hdfs"));
-    assertTrue(mapper.mapPrincipal("kminder").equals("hdfs"));
+    assertTrue(mapper.mapUserPrincipal("lmccay").equals("hdfs"));
+    assertTrue(mapper.mapUserPrincipal("kminder").equals("hdfs"));
     
-    assertTrue(mapper.mapPrincipal("newuser").equals("mapred"));
+    assertTrue(mapper.mapUserPrincipal("newuser").equals("mapred"));
 
-    assertTrue(mapper.mapPrincipal("hdfs").equals("hdfs"));
-    assertTrue(mapper.mapPrincipal("mapred").equals("mapred"));
+    assertTrue(mapper.mapUserPrincipal("hdfs").equals("hdfs"));
+    assertTrue(mapper.mapUserPrincipal("mapred").equals("mapred"));
 
-    assertTrue(mapper.mapPrincipal("stink").equals("stink"));
+    assertTrue(mapper.mapUserPrincipal("stink").equals("stink"));
   }
 
   @Test
   public void testNonNullEndingSemiColonSimplePrincipalMapping() {
     String principalMapping = "lmccay,kminder=hdfs;newuser=mapred;";
     try {
-      mapper.loadMappingTable(principalMapping);
+      mapper.loadMappingTable(principalMapping, null);
     }
     catch (PrincipalMappingException pme) {
       fail();
     }
     
-    assertTrue(mapper.mapPrincipal("lmccay").equals("hdfs"));
-    assertTrue(mapper.mapPrincipal("kminder").equals("hdfs"));
+    assertTrue(mapper.mapUserPrincipal("lmccay").equals("hdfs"));
+    assertTrue(mapper.mapUserPrincipal("kminder").equals("hdfs"));
     
-    assertTrue(mapper.mapPrincipal("newuser").equals("mapred"));
+    assertTrue(mapper.mapUserPrincipal("newuser").equals("mapred"));
 
-    assertTrue(mapper.mapPrincipal("hdfs").equals("hdfs"));
-    assertTrue(mapper.mapPrincipal("mapred").equals("mapred"));
+    assertTrue(mapper.mapUserPrincipal("hdfs").equals("hdfs"));
+    assertTrue(mapper.mapUserPrincipal("mapred").equals("mapred"));
 
-    assertTrue(mapper.mapPrincipal("stink").equals("stink"));
+    assertTrue(mapper.mapUserPrincipal("stink").equals("stink"));
   }
 
   @Test
   public void testNullSimplePrincipalMapping() {
     String principalMapping = null;
     try {
-      mapper.loadMappingTable(principalMapping);
+      mapper.loadMappingTable(principalMapping, null);
     }
     catch (PrincipalMappingException pme) {
       fail();
     }
     
-    assertTrue(mapper.mapPrincipal("lmccay").equals("lmccay"));
-    assertTrue(mapper.mapPrincipal("kminder").equals("kminder"));
+    assertTrue(mapper.mapUserPrincipal("lmccay").equals("lmccay"));
+    assertTrue(mapper.mapUserPrincipal("kminder").equals("kminder"));
     
-    assertTrue(mapper.mapPrincipal("newuser").equals("newuser"));
+    assertTrue(mapper.mapUserPrincipal("newuser").equals("newuser"));
 
-    assertTrue(mapper.mapPrincipal("hdfs").equals("hdfs"));
-    assertTrue(mapper.mapPrincipal("mapred").equals("mapred"));
+    assertTrue(mapper.mapUserPrincipal("hdfs").equals("hdfs"));
+    assertTrue(mapper.mapUserPrincipal("mapred").equals("mapred"));
 
-    assertTrue(mapper.mapPrincipal("stink").equals("stink"));
+    assertTrue(mapper.mapUserPrincipal("stink").equals("stink"));
   }
 
   @Test
   public void testInvalidSimplePrincipalMapping() {
     String principalMapping = "ksdlhfjkdshf;kjdshf";
     try {
-      mapper.loadMappingTable(principalMapping);
+      mapper.loadMappingTable(principalMapping, null);
     }
     catch (PrincipalMappingException pme) {
       // expected
     }
     
-    assertTrue(mapper.mapPrincipal("lmccay").equals("lmccay"));
-    assertTrue(mapper.mapPrincipal("kminder").equals("kminder"));
+    assertTrue(mapper.mapUserPrincipal("lmccay").equals("lmccay"));
+    assertTrue(mapper.mapUserPrincipal("kminder").equals("kminder"));
     
-    assertTrue(mapper.mapPrincipal("newuser").equals("newuser"));
+    assertTrue(mapper.mapUserPrincipal("newuser").equals("newuser"));
 
-    assertTrue(mapper.mapPrincipal("hdfs").equals("hdfs"));
-    assertTrue(mapper.mapPrincipal("mapred").equals("mapred"));
+    assertTrue(mapper.mapUserPrincipal("hdfs").equals("hdfs"));
+    assertTrue(mapper.mapUserPrincipal("mapred").equals("mapred"));
 
-    assertTrue(mapper.mapPrincipal("stink").equals("stink"));
+    assertTrue(mapper.mapUserPrincipal("stink").equals("stink"));
   }
 
   @Test
   public void testPartiallyInvalidSimplePrincipalMapping() {
     String principalMapping = "lmccay=hdfs;kjdshf";
     try {
-      mapper.loadMappingTable(principalMapping);
+      mapper.loadMappingTable(principalMapping, null);
     }
     catch (PrincipalMappingException pme) {
       // expected
     }
     
-    assertTrue(mapper.mapPrincipal("lmccay").equals("lmccay"));
-    assertTrue(mapper.mapPrincipal("kminder").equals("kminder"));
+    assertTrue(mapper.mapUserPrincipal("lmccay").equals("lmccay"));
+    assertTrue(mapper.mapUserPrincipal("kminder").equals("kminder"));
     
-    assertTrue(mapper.mapPrincipal("newuser").equals("newuser"));
+    assertTrue(mapper.mapUserPrincipal("newuser").equals("newuser"));
 
-    assertTrue(mapper.mapPrincipal("hdfs").equals("hdfs"));
-    assertTrue(mapper.mapPrincipal("mapred").equals("mapred"));
+    assertTrue(mapper.mapUserPrincipal("hdfs").equals("hdfs"));
+    assertTrue(mapper.mapUserPrincipal("mapred").equals("mapred"));
 
-    assertTrue(mapper.mapPrincipal("stink").equals("stink"));
+    assertTrue(mapper.mapUserPrincipal("stink").equals("stink"));
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/gateway-test/src/test/java/org/apache/hadoop/gateway/GatewayBasicFuncTest.java
----------------------------------------------------------------------
diff --git a/gateway-test/src/test/java/org/apache/hadoop/gateway/GatewayBasicFuncTest.java b/gateway-test/src/test/java/org/apache/hadoop/gateway/GatewayBasicFuncTest.java
index 27e24a3..9d23187 100644
--- a/gateway-test/src/test/java/org/apache/hadoop/gateway/GatewayBasicFuncTest.java
+++ b/gateway-test/src/test/java/org/apache/hadoop/gateway/GatewayBasicFuncTest.java
@@ -59,6 +59,7 @@ import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.CoreMatchers.not;
 import static org.hamcrest.CoreMatchers.startsWith;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.fail;
 import static org.xmlmatchers.XmlMatchers.isEquivalentTo;
 import static org.xmlmatchers.transform.XmlConverters.the;
 import static uk.co.datumedge.hamcrest.json.SameJSONAs.sameJSONAs;
@@ -177,6 +178,16 @@ public class GatewayBasicFuncTest {
               .addTag( "role" ).addText( "identity-assertion" )
               .addTag( "enabled" ).addText( "true" )
               .addTag( "name" ).addText( "Pseudo" ).gotoParent()
+            .addTag( "provider" )
+              .addTag( "role" ).addText( "authorization" )
+              .addTag( "enabled" ).addText( "true" )
+              .addTag( "name" ).addText( "AclsAuthz" ).gotoParent()
+              .addTag( "param" )
+                .addTag( "name" ).addText( "namenode-acls" )
+                .addTag( "value" ).addText( "hdfs;*;*" ).gotoParent()
+              .addTag( "param" )
+                .addTag( "name" ).addText( "acl.processing.mode" )
+                .addTag( "value" ).addText( "AND" ).gotoParent().gotoParent()
           .gotoRoot()
           .addTag( "service" )
             .addTag( "role" ).addText( "NAMENODE" )
@@ -206,7 +217,6 @@ public class GatewayBasicFuncTest {
     String root = "/tmp/GatewayWebHdfsFuncTest/testBasicJsonUseCase";
     String username = "hdfs";
     String password = "hdfs-password";
-
     /* Create a directory.
     curl -i -X PUT "http://<HOST>:<PORT>/<PATH>?op=MKDIRS[&permission=<OCTAL>]"
 
@@ -362,21 +372,33 @@ public class GatewayBasicFuncTest {
         //.log().all()
         .auth().preemptive().basic( username, "invalid-password" )
         .queryParam( "op", "LISTSTATUS" )
-        .expect()
+    .expect()
         //.log().ifError()
-        .statusCode( HttpStatus.SC_UNAUTHORIZED );
+        .statusCode( HttpStatus.SC_UNAUTHORIZED )
+    .when().get( driver.getUrl( "NAMENODE" ) + root );
     driver.assertComplete();
 
     //NEGATIVE: Test a bad user.
     given()
         //.log().all()
-        .auth().preemptive().basic( "invalid-user", "invalid-password" )
+        .auth().preemptive().basic( "hdfs-user", "hdfs-password" )
         .queryParam( "op", "LISTSTATUS" )
-        .expect()
+    .expect()
         //.log().ifError()
-        .statusCode( HttpStatus.SC_UNAUTHORIZED );
+        .statusCode( HttpStatus.SC_UNAUTHORIZED )
+    .when().get( driver.getUrl( "NAMENODE" ) + root );
     driver.assertComplete();
 
+    //NEGATIVE: Test a valid but unauthorized user.
+    given()
+      //.log().all()
+      .auth().preemptive().basic( "mapred-user", "mapred-password" )
+      .queryParam( "op", "LISTSTATUS" )
+   .expect()
+      //.log().ifError()
+      .statusCode( HttpStatus.SC_UNAUTHORIZED )
+   .when().get( driver.getUrl( "NAMENODE" ) + root );
+
     /* Add a file.
     curl -i -X PUT "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=CREATE
                        [&overwrite=<true|false>][&blocksize=<LONG>][&replication=<SHORT>]

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/660a724f/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index c33aae1..7019946 100644
--- a/pom.xml
+++ b/pom.xml
@@ -44,6 +44,7 @@
         <module>gateway-provider-secure-query</module>
         <module>gateway-provider-security-jwt</module>
         <module>gateway-provider-security-shiro</module>
+        <module>gateway-provider-security-authz-acls</module>
         <module>gateway-provider-identity-assertion-pseudo</module>
         <module>gateway-service-as</module>
         <module>gateway-service-hbase</module>
@@ -315,6 +316,11 @@
             </dependency>
             <dependency>
                 <groupId>${gateway-group}</groupId>
+                <artifactId>gateway-provider-security-authz-acls</artifactId>
+                <version>${gateway-version}</version>
+            </dependency>
+            <dependency>
+                <groupId>${gateway-group}</groupId>
                 <artifactId>gateway-provider-identity-assertion-pseudo</artifactId>
                 <version>${gateway-version}</version>
             </dependency>