You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@knox.apache.org by mo...@apache.org on 2017/02/07 17:58:21 UTC

knox git commit: KNOX-861 - Support for pluggable validator for Header pre authentication provider (Mohammad Kamrul Islam via Sandeep More)

Repository: knox
Updated Branches:
  refs/heads/master 1957648c3 -> 2bdc70394


KNOX-861 - Support for pluggable validator for Header pre authentication provider (Mohammad Kamrul Islam via Sandeep More)


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

Branch: refs/heads/master
Commit: 2bdc703945a3c0645f39dbb35f7165308a8cff80
Parents: 1957648
Author: Sandeep More <mo...@apache.org>
Authored: Tue Feb 7 12:57:48 2017 -0500
Committer: Sandeep More <mo...@apache.org>
Committed: Tue Feb 7 12:57:48 2017 -0500

----------------------------------------------------------------------
 gateway-provider-security-preauth/pom.xml       |  11 ++
 .../filter/AbstractPreAuthFederationFilter.java |  39 ++----
 .../preauth/filter/DefaultValidator.java        |  56 ++++++++
 .../gateway/preauth/filter/IPValidator.java     |  36 +++--
 .../preauth/filter/PreAuthFederationFilter.java |  87 ++++--------
 .../gateway/preauth/filter/PreAuthService.java  |  80 +++++++++++
 .../preauth/filter/PreAuthValidator.java        |  17 ++-
 ...doop.gateway.preauth.filter.PreAuthValidator |  20 +++
 .../federation/DefaultValidatorTest.java        |  43 ++++++
 .../HeaderPreAuthFederationFilterTest.java      | 134 +++++++++++++++++++
 .../provider/federation/IPValidatorTest.java    |  61 +++++++++
 .../provider/federation/PreAuthServiceTest.java |  73 ++++++++++
 ...doop.gateway.preauth.filter.PreAuthValidator |  19 +++
 pom.xml                                         |   6 +
 14 files changed, 580 insertions(+), 102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/knox/blob/2bdc7039/gateway-provider-security-preauth/pom.xml
----------------------------------------------------------------------
diff --git a/gateway-provider-security-preauth/pom.xml b/gateway-provider-security-preauth/pom.xml
index 2256001..2f8abbd 100644
--- a/gateway-provider-security-preauth/pom.xml
+++ b/gateway-provider-security-preauth/pom.xml
@@ -62,6 +62,17 @@
             <scope>test</scope>
         </dependency>
 
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.knox</groupId>
+            <artifactId>gateway-server</artifactId>
+        </dependency>
+
     </dependencies>
 
 </project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/knox/blob/2bdc7039/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/AbstractPreAuthFederationFilter.java
----------------------------------------------------------------------
diff --git a/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/AbstractPreAuthFederationFilter.java b/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/AbstractPreAuthFederationFilter.java
index 63855d9..b40de0a 100644
--- a/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/AbstractPreAuthFederationFilter.java
+++ b/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/AbstractPreAuthFederationFilter.java
@@ -33,6 +33,7 @@ import javax.servlet.ServletResponse;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.gateway.security.PrimaryPrincipal;
 
 /**
@@ -40,10 +41,8 @@ import org.apache.hadoop.gateway.security.PrimaryPrincipal;
  */
 public abstract class AbstractPreAuthFederationFilter implements Filter {
 
-  private static final String VALIDATION_METHOD_PARAM = "preauth.validation.method";
-  private static final String IP_VALIDATION_METHOD_VALUE = "preauth.ip.validation";
-  private static final String IP_ADDRESSES_PARAM = "preauth.ip.addresses";
   private PreAuthValidator validator = null;
+  private FilterConfig filterConfig;
 
   /**
    * 
@@ -54,18 +53,13 @@ public abstract class AbstractPreAuthFederationFilter implements Filter {
 
   @Override
   public void init(FilterConfig filterConfig) throws ServletException {
-    String validationMethod = filterConfig.getInitParameter(VALIDATION_METHOD_PARAM);
-    if (validationMethod != null) {
-      if (IP_VALIDATION_METHOD_VALUE.equals(validationMethod)) {
-        validator = new IPValidator(filterConfig.getInitParameter(IP_ADDRESSES_PARAM));
-      }
-    }
-    else {
-      validator = new DefaultValidator();
-      // TODO: log the fact that there is no verification going on to validate
-      // who is asserting the identity with the a header. Without some validation
-      // we are assuming the network security is the primary protection method.
-    }
+    this.filterConfig = filterConfig;
+    validator = PreAuthService.getValidator(filterConfig);
+  }
+
+  @VisibleForTesting
+  public PreAuthValidator getValidator() {
+    return validator;
   }
 
   @Override
@@ -74,7 +68,7 @@ public abstract class AbstractPreAuthFederationFilter implements Filter {
     HttpServletRequest httpRequest = (HttpServletRequest)request;
     String principal = getPrimaryPrincipal(httpRequest);
     if (principal != null) {
-      if (isValid(httpRequest)) { 
+      if (isValid(httpRequest)) {
         Subject subject = new Subject();
         subject.getPrincipals().add(new PrimaryPrincipal(principal));
         addGroupPrincipals(httpRequest, subject.getPrincipals());
@@ -95,7 +89,7 @@ public abstract class AbstractPreAuthFederationFilter implements Filter {
    */
   private boolean isValid(HttpServletRequest httpRequest) {
     try {
-      return validator.validate(httpRequest);
+      return validator.validate(httpRequest, filterConfig);
     } catch (PreAuthValidationException e) {
       // TODO log exception
       return false;
@@ -142,15 +136,4 @@ public abstract class AbstractPreAuthFederationFilter implements Filter {
    * @param principals
    */
   abstract protected void addGroupPrincipals(HttpServletRequest request, Set<Principal> principals);
-  
-  class DefaultValidator implements PreAuthValidator {
-    /* (non-Javadoc)
-     * @see org.apache.hadoop.gateway.preauth.filter.PreAuthValidator#validate(java.lang.String, java.lang.String)
-     */
-    @Override
-    public boolean validate(HttpServletRequest request)
-        throws PreAuthValidationException {
-      return true;
-    }
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/knox/blob/2bdc7039/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/DefaultValidator.java
----------------------------------------------------------------------
diff --git a/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/DefaultValidator.java b/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/DefaultValidator.java
new file mode 100644
index 0000000..fe1cec5
--- /dev/null
+++ b/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/DefaultValidator.java
@@ -0,0 +1,56 @@
+/**
+ * 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.preauth.filter;
+
+import javax.servlet.FilterConfig;
+import javax.servlet.http.HttpServletRequest;
+
+/**
+ * @since 0.12
+ * This class implements the default Validator where really no validation is performed.
+ * TODO: log the fact that there is no verification going on to validate
+ * +  who is asserting the identity with the a header. Without some validation
+ * +  we are assuming the network security is the primary protection method.
+ */
+public class DefaultValidator implements PreAuthValidator {
+  public static final String DEFAULT_VALIDATION_METHOD_VALUE = "preauth.default.validation";
+
+  public DefaultValidator() {
+  }
+
+  /**
+   * @param httpRequest
+   * @param filterConfig
+   * @return true if validated, otherwise false
+   * @throws PreAuthValidationException
+   */
+  @Override
+  public boolean validate(HttpServletRequest httpRequest, FilterConfig filterConfig) throws PreAuthValidationException {
+    return true;
+  }
+
+  /**
+   * Return unique validator name
+   *
+   * @return name of validator
+   */
+  @Override
+  public String getName() {
+    return DEFAULT_VALIDATION_METHOD_VALUE;
+  }
+}

http://git-wip-us.apache.org/repos/asf/knox/blob/2bdc7039/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/IPValidator.java
----------------------------------------------------------------------
diff --git a/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/IPValidator.java b/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/IPValidator.java
index d19ee58..9df23b5 100644
--- a/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/IPValidator.java
+++ b/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/IPValidator.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.gateway.preauth.filter;
 
+import javax.servlet.FilterConfig;
 import javax.servlet.http.HttpServletRequest;
 
 import org.apache.hadoop.gateway.util.IpAddressValidator;
@@ -24,23 +25,34 @@ import org.apache.hadoop.gateway.util.IpAddressValidator;
 /**
  *
  */
-class IPValidator implements PreAuthValidator {
-  private IpAddressValidator ipv = null;
-  
+public class IPValidator implements PreAuthValidator {
+  public static final String IP_ADDRESSES_PARAM = "preauth.ip.addresses";
+  public static final String IP_VALIDATION_METHOD_VALUE = "preauth.ip.validation";
+
+  public IPValidator() {
+  }
+
   /**
-   * @param initParameter
+   * @param httpRequest
+   * @param filterConfig
+   * @return true if validated, otherwise false
+   * @throws PreAuthValidationException
    */
-  public IPValidator(String ipParam) {
-    ipv = new IpAddressValidator(ipParam);
+  @Override
+  public boolean validate(HttpServletRequest httpRequest, FilterConfig filterConfig)
+      throws PreAuthValidationException {
+    String ipParam = filterConfig.getInitParameter(IP_ADDRESSES_PARAM);
+    IpAddressValidator ipv = new IpAddressValidator(ipParam);
+    return ipv.validateIpAddress(httpRequest.getRemoteAddr());
   }
 
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.gateway.preauth.filter.PreAuthValidator#validate(java.lang.String, java.lang.String)
+  /**
+   * Return unique validator name
+   *
+   * @return name of validator
    */
   @Override
-  public boolean validate(HttpServletRequest request)
-      throws PreAuthValidationException {
-    
-    return ipv.validateIpAddress(request.getRemoteAddr());
+  public String getName() {
+    return IP_VALIDATION_METHOD_VALUE;
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/knox/blob/2bdc7039/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/PreAuthFederationFilter.java
----------------------------------------------------------------------
diff --git a/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/PreAuthFederationFilter.java b/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/PreAuthFederationFilter.java
index 4c0bb10..63eb411 100644
--- a/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/PreAuthFederationFilter.java
+++ b/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/PreAuthFederationFilter.java
@@ -37,48 +37,34 @@ import org.apache.hadoop.gateway.security.PrimaryPrincipal;
 
 public class PreAuthFederationFilter implements Filter {
   private static final String CUSTOM_HEADER_PARAM = "preauth.customHeader";
-  private static final String VALIDATION_METHOD_PARAM = "preauth.validation.method";
-  private static final String IP_VALIDATION_METHOD_VALUE = "preauth.ip.validation";
-  private static final String IP_ADDRESSES_PARAM = "preauth.ip.addresses";
   private PreAuthValidator validator = null;
-  private String  headerName = "SM_USER";
-  
+  private FilterConfig filterConfig;
+  private String headerName = "SM_USER";
+
   @Override
-  public void init( FilterConfig filterConfig ) throws ServletException {
+  public void init(FilterConfig filterConfig) throws ServletException {
     String customHeader = filterConfig.getInitParameter(CUSTOM_HEADER_PARAM);
     if (customHeader != null) {
       headerName = customHeader;
     }
-    String validationMethod = filterConfig.getInitParameter(VALIDATION_METHOD_PARAM);
-    if (validationMethod != null) {
-      if (IP_VALIDATION_METHOD_VALUE.equals(validationMethod)) {
-        validator = new IPValidator(filterConfig.getInitParameter(IP_ADDRESSES_PARAM));
-      }
-    }
-    else {
-      validator = new DefaultValidator();
-      // TODO: log the fact that there is no verification going on to validate
-      // who is asserting the identity with the a header. Without some validation
-      // we are assuming the network security is the primary protection method.
-    }
+    this.filterConfig = filterConfig;
+    validator = PreAuthService.getValidator(filterConfig);
   }
-  
+
   @Override
   public void doFilter(ServletRequest request, ServletResponse response,
-      FilterChain chain) throws IOException, ServletException {
-    HttpServletRequest httpRequest = (HttpServletRequest)request;
+                       FilterChain chain) throws IOException, ServletException {
+    HttpServletRequest httpRequest = (HttpServletRequest) request;
     if (httpRequest.getHeader(headerName) != null) {
-      if (isValid(httpRequest)) { 
+      if (isValid(httpRequest)) {
         // TODO: continue as subject
         chain.doFilter(request, response);
-      }
-      else {
+      } else {
         // TODO: log preauthenticated SSO validation failure
-        ((HttpServletResponse)response).sendError(HttpServletResponse.SC_BAD_REQUEST, "Missing Required Header for SSO Validation");
+        ((HttpServletResponse) response).sendError(HttpServletResponse.SC_BAD_REQUEST, "Missing Required Header for SSO Validation");
       }
-    } 
-    else {
-      ((HttpServletResponse)response).sendError(HttpServletResponse.SC_BAD_REQUEST, "Missing Required Header for PreAuth SSO Federation");
+    } else {
+      ((HttpServletResponse) response).sendError(HttpServletResponse.SC_BAD_REQUEST, "Missing Required Header for PreAuth SSO Federation");
     }
   }
 
@@ -87,7 +73,7 @@ public class PreAuthFederationFilter implements Filter {
    */
   private boolean isValid(HttpServletRequest httpRequest) {
     try {
-      return validator.validate(httpRequest);
+      return validator.validate(httpRequest, filterConfig);
     } catch (PreAuthValidationException e) {
       // TODO log exception
       return false;
@@ -100,26 +86,26 @@ public class PreAuthFederationFilter implements Filter {
   @Override
   public void destroy() {
     // TODO Auto-generated method stub
-    
+
   }
-  
+
   /**
    * 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 
+   * @throws ServletException
+   * @throws IOException
    */
-  protected void continueChainAsPrincipal(final ServletRequest request, final ServletResponse response, 
-      final FilterChain chain, String principal) throws IOException, ServletException {
+  protected void continueChainAsPrincipal(final ServletRequest request, final ServletResponse response,
+                                          final FilterChain chain, String principal) throws IOException, ServletException {
     Subject subject = null;
     Principal primaryPrincipal = null;
-    
+
     // do some check to ensure that the extracted identity matches any existing security context
     // if not, there is may be someone tampering with the request - consult config to determine
     // how we are to handle it
-    
+
     // TODO: make sure that this makes sense with existing sessions or lack thereof
     Subject currentSubject = Subject.getSubject(AccessController.getContext());
     if (currentSubject != null) {
@@ -129,14 +115,14 @@ public class PreAuthFederationFilter implements Filter {
         }
       }
     }
-    
+
     subject = new Subject();
     subject.getPrincipals().add(primaryPrincipal);
     doAs(request, response, chain, subject);
   }
 
   private void doAs(final ServletRequest request,
-      final ServletResponse response, final FilterChain chain, Subject subject)
+                    final ServletResponse response, final FilterChain chain, Subject subject)
       throws IOException, ServletException {
     try {
       Subject.doAs(
@@ -147,17 +133,14 @@ public class PreAuthFederationFilter implements Filter {
               return null;
             }
           }
-          );
-    }
-    catch (PrivilegedActionException e) {
+      );
+    } catch (PrivilegedActionException e) {
       Throwable t = e.getCause();
       if (t instanceof IOException) {
         throw (IOException) t;
-      }
-      else if (t instanceof ServletException) {
+      } else if (t instanceof ServletException) {
         throw (ServletException) t;
-      }
-      else {
+      } else {
         throw new ServletException(t);
       }
     }
@@ -166,17 +149,5 @@ public class PreAuthFederationFilter implements Filter {
   private void doFilterInternal(ServletRequest request, ServletResponse response, FilterChain chain) throws IOException, ServletException {
     chain.doFilter(request, response);
   }
-  
-  class DefaultValidator implements PreAuthValidator {
-    /* (non-Javadoc)
-     * @see org.apache.hadoop.gateway.preauth.filter.PreAuthValidator#validate(java.lang.String, java.lang.String)
-     */
-    @Override
-    public boolean validate(HttpServletRequest request)
-        throws PreAuthValidationException {
-      return true;
-    }
-    
-  }
 
 }

http://git-wip-us.apache.org/repos/asf/knox/blob/2bdc7039/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/PreAuthService.java
----------------------------------------------------------------------
diff --git a/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/PreAuthService.java b/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/PreAuthService.java
new file mode 100644
index 0000000..778ee9d
--- /dev/null
+++ b/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/PreAuthService.java
@@ -0,0 +1,80 @@
+/**
+ * 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.preauth.filter;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+import java.util.Set;
+import java.util.Collections;
+import java.util.ServiceLoader;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class manages few utility methods used across different classes of pre-auth module
+ * @since 0.12
+ */
+public class PreAuthService {
+
+  public static final String VALIDATION_METHOD_PARAM = "preauth.validation.method";
+  private static ConcurrentHashMap<String, PreAuthValidator> validatorMap;
+
+  static {
+    initializeValidators();
+  }
+
+
+  private static void initializeValidators() {
+    ServiceLoader<PreAuthValidator> servLoader = ServiceLoader.load(PreAuthValidator.class);
+    validatorMap = new ConcurrentHashMap<>();
+    for (Iterator<PreAuthValidator> iterator = servLoader.iterator(); iterator.hasNext(); ) {
+      PreAuthValidator validator = iterator.next();
+      validatorMap.put(validator.getName(), validator);
+    }
+  }
+
+  @VisibleForTesting
+  public static Map<String, PreAuthValidator> getValidatorMap() {
+    return Collections.unmodifiableMap(validatorMap);
+  }
+
+  /**
+   * This method returns appropriate pre-auth Validator as defined in config
+   *
+   * @since 0.12
+   * @param filterConfig
+   * @return PreAuthValidator
+   * @throws ServletException
+   */
+  public static PreAuthValidator getValidator(FilterConfig filterConfig) throws ServletException {
+    String validationMethod = filterConfig.getInitParameter(VALIDATION_METHOD_PARAM);
+    if (Strings.isNullOrEmpty(validationMethod)) {
+      validationMethod = DefaultValidator.DEFAULT_VALIDATION_METHOD_VALUE;
+    }
+    if (validatorMap.containsKey(validationMethod)) {
+      return validatorMap.get(validationMethod);
+    } else {
+      throw new ServletException(String.format("Unable to find validator with name '%s'", validationMethod));
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/knox/blob/2bdc7039/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/PreAuthValidator.java
----------------------------------------------------------------------
diff --git a/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/PreAuthValidator.java b/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/PreAuthValidator.java
index 7013259..5819801 100644
--- a/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/PreAuthValidator.java
+++ b/gateway-provider-security-preauth/src/main/java/org/apache/hadoop/gateway/preauth/filter/PreAuthValidator.java
@@ -17,17 +17,26 @@
  */
 package org.apache.hadoop.gateway.preauth.filter;
 
+import javax.servlet.FilterConfig;
 import javax.servlet.http.HttpServletRequest;
 
 /**
  *
  */
 public interface PreAuthValidator {
-
   /**
    * @param httpRequest
-   * @return
-   * @throws PreAuthValidationException 
+   * @param filterConfig
+   * @return true if validated, otherwise false
+   * @throws PreAuthValidationException
+   */
+  public abstract boolean validate(HttpServletRequest httpRequest, FilterConfig filterConfig) throws
+      PreAuthValidationException;
+
+  /**
+   * Return unique validator name
+   *
+   * @return name of validator
    */
-  boolean validate(HttpServletRequest httpRequest) throws PreAuthValidationException;
+  public abstract String getName();
 }

http://git-wip-us.apache.org/repos/asf/knox/blob/2bdc7039/gateway-provider-security-preauth/src/main/resources/META-INF/services/org.apache.hadoop.gateway.preauth.filter.PreAuthValidator
----------------------------------------------------------------------
diff --git a/gateway-provider-security-preauth/src/main/resources/META-INF/services/org.apache.hadoop.gateway.preauth.filter.PreAuthValidator b/gateway-provider-security-preauth/src/main/resources/META-INF/services/org.apache.hadoop.gateway.preauth.filter.PreAuthValidator
new file mode 100644
index 0000000..808dbe8
--- /dev/null
+++ b/gateway-provider-security-preauth/src/main/resources/META-INF/services/org.apache.hadoop.gateway.preauth.filter.PreAuthValidator
@@ -0,0 +1,20 @@
+##########################################################################
+# 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.preauth.filter.IPValidator
+org.apache.hadoop.gateway.preauth.filter.DefaultValidator
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/knox/blob/2bdc7039/gateway-provider-security-preauth/src/test/java/org/apache/hadoop/gateway/provider/federation/DefaultValidatorTest.java
----------------------------------------------------------------------
diff --git a/gateway-provider-security-preauth/src/test/java/org/apache/hadoop/gateway/provider/federation/DefaultValidatorTest.java b/gateway-provider-security-preauth/src/test/java/org/apache/hadoop/gateway/provider/federation/DefaultValidatorTest.java
new file mode 100644
index 0000000..4096b48
--- /dev/null
+++ b/gateway-provider-security-preauth/src/test/java/org/apache/hadoop/gateway/provider/federation/DefaultValidatorTest.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.gateway.provider.federation;
+
+import junit.framework.TestCase;
+import org.apache.hadoop.gateway.preauth.filter.DefaultValidator;
+import org.junit.Test;
+
+import static org.mockito.Mockito.mock;
+
+import javax.servlet.FilterConfig;
+import javax.servlet.http.HttpServletRequest;
+
+public class DefaultValidatorTest extends TestCase {
+  @Test
+  public void testDefault() throws Exception {
+    final FilterConfig filterConfig = mock(FilterConfig.class);
+    final HttpServletRequest request = mock(HttpServletRequest.class);
+    DefaultValidator dv = new DefaultValidator();
+    assertTrue(dv.validate(request, filterConfig));
+  }
+
+  @Test
+  public void testName() {
+    DefaultValidator dv = new DefaultValidator();
+    assertEquals(dv.getName(), DefaultValidator.DEFAULT_VALIDATION_METHOD_VALUE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/knox/blob/2bdc7039/gateway-provider-security-preauth/src/test/java/org/apache/hadoop/gateway/provider/federation/HeaderPreAuthFederationFilterTest.java
----------------------------------------------------------------------
diff --git a/gateway-provider-security-preauth/src/test/java/org/apache/hadoop/gateway/provider/federation/HeaderPreAuthFederationFilterTest.java b/gateway-provider-security-preauth/src/test/java/org/apache/hadoop/gateway/provider/federation/HeaderPreAuthFederationFilterTest.java
new file mode 100644
index 0000000..28d7eeb
--- /dev/null
+++ b/gateway-provider-security-preauth/src/test/java/org/apache/hadoop/gateway/provider/federation/HeaderPreAuthFederationFilterTest.java
@@ -0,0 +1,134 @@
+/**
+ * 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.provider.federation;
+
+import junit.framework.TestCase;
+import org.apache.hadoop.gateway.preauth.filter.*;
+import org.junit.Test;
+
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class HeaderPreAuthFederationFilterTest extends TestCase {
+
+  @Test
+  public void testDefaultValidator() throws ServletException, PreAuthValidationException {
+    HeaderPreAuthFederationFilter hpaff = new HeaderPreAuthFederationFilter();
+    final HttpServletRequest request = mock(HttpServletRequest.class);
+    final FilterConfig filterConfig = mock(FilterConfig.class);
+    when(filterConfig.getInitParameter(PreAuthService.VALIDATION_METHOD_PARAM)).thenReturn
+        (DefaultValidator.DEFAULT_VALIDATION_METHOD_VALUE);
+    hpaff.init(filterConfig);
+    PreAuthValidator validator = hpaff.getValidator();
+    assertEquals(validator.getName(), DefaultValidator.DEFAULT_VALIDATION_METHOD_VALUE);
+    assertTrue(validator.validate(request, filterConfig));
+  }
+
+  @Test
+  public void testIPValidator() throws ServletException, PreAuthValidationException {
+    HeaderPreAuthFederationFilter hpaff = new HeaderPreAuthFederationFilter();
+    final HttpServletRequest request = mock(HttpServletRequest.class);
+    when(request.getRemoteAddr()).thenReturn("10.1.23.42");
+    final FilterConfig filterConfig = mock(FilterConfig.class);
+    when(filterConfig.getInitParameter(IPValidator.IP_ADDRESSES_PARAM)).thenReturn("5.4.3.2,10.1.23.42");
+    when(filterConfig.getInitParameter(PreAuthService.VALIDATION_METHOD_PARAM)).thenReturn(IPValidator
+        .IP_VALIDATION_METHOD_VALUE);
+    hpaff.init(filterConfig);
+    PreAuthValidator validator = hpaff.getValidator();
+    assertEquals(validator.getName(), IPValidator.IP_VALIDATION_METHOD_VALUE);
+    assertTrue(validator.validate(request, filterConfig));
+    //Negative testing
+    when(request.getRemoteAddr()).thenReturn("10.10.22.33");
+    assertFalse(validator.validate(request, filterConfig));
+  }
+
+  @Test
+  public void testCustomValidatorPositive() throws ServletException, PreAuthValidationException {
+    HeaderPreAuthFederationFilter hpaff = new HeaderPreAuthFederationFilter();
+    final HttpServletRequest request = mock(HttpServletRequest.class);
+    final FilterConfig filterConfig = mock(FilterConfig.class);
+    when(filterConfig.getInitParameter(PreAuthService.VALIDATION_METHOD_PARAM)).thenReturn
+        (DummyValidator.NAME);
+
+    hpaff.init(filterConfig);
+    PreAuthValidator validator = hpaff.getValidator();
+    assertEquals(validator.getName(), DummyValidator.NAME);
+    //Positive test
+    when(request.getHeader("CUSTOM_TOKEN")).thenReturn("HelloWorld");
+    assertTrue(validator.validate(request, filterConfig));
+
+  }
+
+  @Test
+  public void testCustomValidatorNegative() throws ServletException, PreAuthValidationException {
+    HeaderPreAuthFederationFilter hpaff = new HeaderPreAuthFederationFilter();
+    final HttpServletRequest request = mock(HttpServletRequest.class);
+    final FilterConfig filterConfig = mock(FilterConfig.class);
+    when(filterConfig.getInitParameter(PreAuthService.VALIDATION_METHOD_PARAM)).thenReturn
+        (DummyValidator.NAME);
+
+    hpaff.init(filterConfig);
+    PreAuthValidator validator = hpaff.getValidator();
+    assertEquals(validator.getName(), DummyValidator.NAME);
+
+    when(request.getHeader("CUSTOM_TOKEN")).thenReturn("NOTHelloWorld");
+    assertFalse(validator.validate(request, filterConfig));
+
+  }
+
+
+  public static class DummyValidator implements PreAuthValidator {
+    public static String NAME = "DummyValidator";
+
+    public DummyValidator() {
+
+    }
+
+    /**
+     * @param httpRequest
+     * @param filterConfig
+     * @return true if validated, otherwise false
+     * @throws PreAuthValidationException
+     */
+    @Override
+    public boolean validate(HttpServletRequest httpRequest, FilterConfig filterConfig) throws
+        PreAuthValidationException {
+      String token = httpRequest.getHeader("CUSTOM_TOKEN");
+      if (token.equalsIgnoreCase("HelloWorld")) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+
+    /**
+     * Return unique validator name
+     *
+     * @return name of validator
+     */
+    @Override
+    public String getName() {
+      return NAME;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/knox/blob/2bdc7039/gateway-provider-security-preauth/src/test/java/org/apache/hadoop/gateway/provider/federation/IPValidatorTest.java
----------------------------------------------------------------------
diff --git a/gateway-provider-security-preauth/src/test/java/org/apache/hadoop/gateway/provider/federation/IPValidatorTest.java b/gateway-provider-security-preauth/src/test/java/org/apache/hadoop/gateway/provider/federation/IPValidatorTest.java
new file mode 100644
index 0000000..23c0096
--- /dev/null
+++ b/gateway-provider-security-preauth/src/test/java/org/apache/hadoop/gateway/provider/federation/IPValidatorTest.java
@@ -0,0 +1,61 @@
+/**
+ * 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.provider.federation;
+
+import junit.framework.TestCase;
+import org.apache.hadoop.gateway.preauth.filter.IPValidator;
+import org.apache.hadoop.gateway.preauth.filter.PreAuthValidationException;
+import org.junit.Test;
+
+import javax.servlet.FilterConfig;
+import javax.servlet.http.HttpServletRequest;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class IPValidatorTest extends TestCase {
+
+  @Test
+  public void testName() {
+    IPValidator ipv = new IPValidator();
+    assertEquals(ipv.getName(), IPValidator.IP_VALIDATION_METHOD_VALUE);
+  }
+
+
+  @Test
+  public void testIPAddressPositive() throws PreAuthValidationException {
+    IPValidator ipv = new IPValidator();
+    final HttpServletRequest request = mock(HttpServletRequest.class);
+    when(request.getRemoteAddr()).thenReturn("10.1.23.42");
+    final FilterConfig filterConfig = mock(FilterConfig.class);
+    when(filterConfig.getInitParameter(IPValidator.IP_ADDRESSES_PARAM)).thenReturn("5.4.3.2,10.1.23.42");
+    assertTrue(ipv.validate(request, filterConfig));
+  }
+
+  @Test
+  public void testIPAddressNegative() throws PreAuthValidationException {
+    IPValidator ipv = new IPValidator();
+    final HttpServletRequest request = mock(HttpServletRequest.class);
+    when(request.getRemoteAddr()).thenReturn("10.1.23.42");
+    final FilterConfig filterConfig = mock(FilterConfig.class);
+    when(filterConfig.getInitParameter(IPValidator.IP_ADDRESSES_PARAM)).thenReturn("10.22.34.56");
+    assertFalse(ipv.validate(request, filterConfig));
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/knox/blob/2bdc7039/gateway-provider-security-preauth/src/test/java/org/apache/hadoop/gateway/provider/federation/PreAuthServiceTest.java
----------------------------------------------------------------------
diff --git a/gateway-provider-security-preauth/src/test/java/org/apache/hadoop/gateway/provider/federation/PreAuthServiceTest.java b/gateway-provider-security-preauth/src/test/java/org/apache/hadoop/gateway/provider/federation/PreAuthServiceTest.java
new file mode 100644
index 0000000..8186189
--- /dev/null
+++ b/gateway-provider-security-preauth/src/test/java/org/apache/hadoop/gateway/provider/federation/PreAuthServiceTest.java
@@ -0,0 +1,73 @@
+/**
+ * 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.provider.federation;
+
+import junit.framework.TestCase;
+import org.apache.hadoop.gateway.preauth.filter.*;
+import org.junit.Test;
+
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+
+import java.util.Map;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class PreAuthServiceTest extends TestCase {
+
+  @Test
+  public void testValidatorMap() {
+    Map<String, PreAuthValidator> valMap = PreAuthService.getValidatorMap();
+    assertNotNull(valMap.get(IPValidator.IP_VALIDATION_METHOD_VALUE));
+    assertEquals(valMap.get(IPValidator.IP_VALIDATION_METHOD_VALUE).getName(), IPValidator.IP_VALIDATION_METHOD_VALUE);
+    assertNotNull(valMap.get(DefaultValidator.DEFAULT_VALIDATION_METHOD_VALUE));
+    assertEquals(valMap.get(DefaultValidator.DEFAULT_VALIDATION_METHOD_VALUE).getName(), DefaultValidator.DEFAULT_VALIDATION_METHOD_VALUE);
+
+    //Negative test
+    assertNull(valMap.get("NonExists"));
+  }
+
+  @Test
+  public void testDefaultValidator() throws ServletException, PreAuthValidationException {
+    final HttpServletRequest request = mock(HttpServletRequest.class);
+    final FilterConfig filterConfig = mock(FilterConfig.class);
+    when(filterConfig.getInitParameter(PreAuthService.VALIDATION_METHOD_PARAM)).thenReturn
+        (DefaultValidator.DEFAULT_VALIDATION_METHOD_VALUE);
+    PreAuthValidator validator = PreAuthService.getValidator(filterConfig);
+    assertEquals(validator.getName(), DefaultValidator.DEFAULT_VALIDATION_METHOD_VALUE);
+    assertTrue(validator.validate(request, filterConfig));
+  }
+
+  @Test
+  public void testIPValidator() throws ServletException, PreAuthValidationException {
+    final HttpServletRequest request = mock(HttpServletRequest.class);
+    when(request.getRemoteAddr()).thenReturn("10.1.23.42");
+    final FilterConfig filterConfig = mock(FilterConfig.class);
+    when(filterConfig.getInitParameter(IPValidator.IP_ADDRESSES_PARAM)).thenReturn("5.4.3.2,10.1.23.42");
+    when(filterConfig.getInitParameter(PreAuthService.VALIDATION_METHOD_PARAM)).thenReturn(IPValidator
+        .IP_VALIDATION_METHOD_VALUE);
+    PreAuthValidator validator = PreAuthService.getValidator(filterConfig);
+    assertEquals(validator.getName(), IPValidator.IP_VALIDATION_METHOD_VALUE);
+    assertTrue(validator.validate(request, filterConfig));
+    //Negative testing
+    when(request.getRemoteAddr()).thenReturn("10.10.22.33");
+    assertFalse(validator.validate(request, filterConfig));
+  }
+}

http://git-wip-us.apache.org/repos/asf/knox/blob/2bdc7039/gateway-provider-security-preauth/src/test/resources/META-INF/services/org.apache.hadoop.gateway.preauth.filter.PreAuthValidator
----------------------------------------------------------------------
diff --git a/gateway-provider-security-preauth/src/test/resources/META-INF/services/org.apache.hadoop.gateway.preauth.filter.PreAuthValidator b/gateway-provider-security-preauth/src/test/resources/META-INF/services/org.apache.hadoop.gateway.preauth.filter.PreAuthValidator
new file mode 100644
index 0000000..911bd0f
--- /dev/null
+++ b/gateway-provider-security-preauth/src/test/resources/META-INF/services/org.apache.hadoop.gateway.preauth.filter.PreAuthValidator
@@ -0,0 +1,19 @@
+##########################################################################
+# 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.provider.federation.HeaderPreAuthFederationFilterTest$DummyValidator
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/knox/blob/2bdc7039/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1e1e830..a5393ea 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1264,6 +1264,12 @@
                 <version>3.3</version>
                 <scope>test</scope>
             </dependency>
+            <dependency>
+                <groupId>org.mockito</groupId>
+                <artifactId>mockito-core</artifactId>
+                <version>1.10.19</version>
+                <scope>test</scope>
+            </dependency>
 
             <dependency>
                 <groupId>org.xmlmatchers</groupId>