You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ct...@apache.org on 2016/09/23 19:50:50 UTC

[2/2] hive git commit: HIVE-14713: LDAP Authentication Provider should be covered with unit tests (Illya Yalovyy, reviewed by Chaoyu Tang, Szehon Ho)

HIVE-14713: LDAP Authentication Provider should be covered with unit tests (Illya Yalovyy, reviewed by Chaoyu Tang, Szehon Ho)


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

Branch: refs/heads/master
Commit: 990927e3dcddcc7c82a16437d55d9f7ea9a1a447
Parents: 421d97a
Author: ctang <ct...@cloudera.com>
Authored: Fri Sep 23 15:50:32 2016 -0400
Committer: ctang <ct...@cloudera.com>
Committed: Fri Sep 23 15:50:32 2016 -0400

----------------------------------------------------------------------
 service/pom.xml                                 |   7 +
 .../auth/LdapAuthenticationProviderImpl.java    | 657 ++-----------------
 .../service/auth/ldap/ChainFilterFactory.java   |  78 +++
 .../auth/ldap/CustomQueryFilterFactory.java     |  84 +++
 .../hive/service/auth/ldap/DirSearch.java       |  52 ++
 .../service/auth/ldap/DirSearchFactory.java     |  37 ++
 .../apache/hive/service/auth/ldap/Filter.java   |  36 +
 .../hive/service/auth/ldap/FilterFactory.java   |  33 +
 .../service/auth/ldap/GroupFilterFactory.java   |  90 +++
 .../hive/service/auth/ldap/LdapSearch.java      | 155 +++++
 .../service/auth/ldap/LdapSearchFactory.java    |  64 ++
 .../hive/service/auth/ldap/LdapUtils.java       | 228 +++++++
 .../apache/hive/service/auth/ldap/Query.java    | 154 +++++
 .../hive/service/auth/ldap/QueryFactory.java    | 135 ++++
 .../service/auth/ldap/SearchResultHandler.java  | 163 +++++
 .../service/auth/ldap/UserFilterFactory.java    |  75 +++
 .../auth/ldap/UserSearchFilterFactory.java      |  65 ++
 .../auth/TestLdapAtnProviderWithMiniDS.java     |   3 +-
 .../TestLdapAuthenticationProviderImpl.java     | 277 +++++++-
 .../hive/service/auth/ldap/Credentials.java     |  41 ++
 .../hive/service/auth/ldap/LdapTestUtils.java   | 126 ++++
 .../hive/service/auth/ldap/TestChainFilter.java | 103 +++
 .../auth/ldap/TestCustomQueryFilter.java        |  85 +++
 .../hive/service/auth/ldap/TestGroupFilter.java | 101 +++
 .../hive/service/auth/ldap/TestLdapSearch.java  | 209 ++++++
 .../hive/service/auth/ldap/TestLdapUtils.java   | 103 +++
 .../hive/service/auth/ldap/TestQuery.java       |  59 ++
 .../service/auth/ldap/TestQueryFactory.java     |  79 +++
 .../auth/ldap/TestSearchResultHandler.java      | 222 +++++++
 .../hive/service/auth/ldap/TestUserFilter.java  |  75 +++
 .../service/auth/ldap/TestUserSearchFilter.java |  94 +++
 31 files changed, 3062 insertions(+), 628 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/pom.xml
----------------------------------------------------------------------
diff --git a/service/pom.xml b/service/pom.xml
index ecea719..9306739 100644
--- a/service/pom.xml
+++ b/service/pom.xml
@@ -164,6 +164,13 @@
     </dependency>
 
     <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <version>${mockito-all.version}</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
       <groupId>org.apache.directory.client.ldap</groupId>
       <artifactId>ldap-client-api</artifactId>
       <version>${apache-directory-clientapi.version}</version>

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java b/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java
index efd5393..c21da28 100644
--- a/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java
+++ b/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java
@@ -17,633 +17,106 @@
  */
 package org.apache.hive.service.auth;
 
-import java.util.ArrayList;
-import java.util.Hashtable;
-import java.util.List;
-import java.util.ListIterator;
-
-import javax.naming.Context;
-import javax.naming.NamingEnumeration;
-import javax.naming.NamingException;
-import javax.naming.directory.Attribute;
-import javax.naming.directory.Attributes;
-import javax.naming.directory.DirContext;
-import javax.naming.directory.InitialDirContext;
-import javax.naming.directory.SearchControls;
-import javax.naming.directory.SearchResult;
 import javax.security.sasl.AuthenticationException;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hive.service.ServiceUtils;
+import org.apache.hive.service.auth.ldap.ChainFilterFactory;
+import org.apache.hive.service.auth.ldap.CustomQueryFilterFactory;
+import org.apache.hive.service.auth.ldap.LdapSearchFactory;
+import org.apache.hive.service.auth.ldap.Filter;
+import org.apache.hive.service.auth.ldap.DirSearch;
+import org.apache.hive.service.auth.ldap.DirSearchFactory;
+import org.apache.hive.service.auth.ldap.FilterFactory;
+import org.apache.hive.service.auth.ldap.GroupFilterFactory;
+import org.apache.hive.service.auth.ldap.LdapUtils;
+import org.apache.hive.service.auth.ldap.UserFilterFactory;
+import org.apache.hive.service.auth.ldap.UserSearchFilterFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class LdapAuthenticationProviderImpl implements PasswdAuthenticationProvider {
 
-  private static final Logger LOG     = LoggerFactory.getLogger(LdapAuthenticationProviderImpl.class);
+  private static final Logger LOG = LoggerFactory.getLogger(LdapAuthenticationProviderImpl.class);
 
-  private String ldapURL;
-  private String baseDN;
-  private String ldapDomain;
-  private static List<String> groupBases;
-  private static List<String> userBases;
-  private static List<String> userFilter;
-  private static List<String> groupFilter;
-  private String customQuery;
-  private static String guid_attr;
-  private static String groupMembership_attr;
-  private static String groupClass_attr;
-
-  LdapAuthenticationProviderImpl(HiveConf conf) {
-    init(conf);
-  }
-
-  protected void init(HiveConf conf) {
-    ldapURL     = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_URL);
-    baseDN      = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_BASEDN);
-    ldapDomain  = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_DOMAIN);
-    customQuery = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_CUSTOMLDAPQUERY);
-    guid_attr   = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GUIDKEY);
-    groupBases  = new ArrayList<String>();
-    userBases   = new ArrayList<String>();
-    userFilter  = new ArrayList<String>();
-    groupFilter = new ArrayList<String>();
-
-    String groupDNPatterns = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPDNPATTERN);
-    String groupFilterVal  = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPFILTER);
-    String userDNPatterns  = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERDNPATTERN);
-    String userFilterVal   = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER);
-    groupMembership_attr   = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPMEMBERSHIP_KEY);
-    groupClass_attr        = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPCLASS_KEY);
-
-    // parse COLON delimited root DNs for users/groups that may or may not be under BaseDN.
-    // Expect the root DNs be fully qualified including the baseDN
-    if (groupDNPatterns != null && groupDNPatterns.trim().length() > 0) {
-      String[] groupTokens = groupDNPatterns.split(":");
-      for (int i = 0; i < groupTokens.length; i++) {
-        if (groupTokens[i].contains(",") && groupTokens[i].contains("=")) {
-          groupBases.add(groupTokens[i]);
-        } else {
-          LOG.warn("Unexpected format for " + HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPDNPATTERN
-                       + "..ignoring " + groupTokens[i]);
-        }
-      }
-    } else if (baseDN != null) {
-      groupBases.add(guid_attr + "=%s," + baseDN);
-    }
+  private static final List<FilterFactory> FILTER_FACTORIES = ImmutableList.<FilterFactory>of(
+      new CustomQueryFilterFactory(),
+      new ChainFilterFactory(new UserSearchFilterFactory(), new UserFilterFactory(),
+          new GroupFilterFactory())
+  );
 
-    if (groupFilterVal != null && groupFilterVal.trim().length() > 0) {
-      String[] groups = groupFilterVal.split(",");
-      for (int i = 0; i < groups.length; i++) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Filtered group: " + groups[i]);
-        }
-        groupFilter.add(groups[i]);
-      }
-    }
+  private final HiveConf conf;
+  private final Filter filter;
+  private final DirSearchFactory searchFactory;
 
-    if (userDNPatterns != null && userDNPatterns.trim().length() > 0) {
-      String[] userTokens = userDNPatterns.split(":");
-      for (int i = 0; i < userTokens.length; i++) {
-        if (userTokens[i].contains(",") && userTokens[i].contains("=")) {
-          userBases.add(userTokens[i]);
-        } else {
-          LOG.warn("Unexpected format for " + HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERDNPATTERN
-                       + "..ignoring " + userTokens[i]);
-        }
-      }
-    } else if (baseDN != null) {
-      userBases.add(guid_attr + "=%s," + baseDN);
-    }
+  public LdapAuthenticationProviderImpl(HiveConf conf) {
+    this(conf, new LdapSearchFactory());
+  }
 
-    if (userFilterVal != null && userFilterVal.trim().length() > 0) {
-      String[] users = userFilterVal.split(",");
-      for (int i = 0; i < users.length; i++) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Filtered user: " + users[i]);
-        }
-        userFilter.add(users[i]);
-      }
-    }
+  @VisibleForTesting
+  LdapAuthenticationProviderImpl(HiveConf conf, DirSearchFactory searchFactory) {
+    this.conf = conf;
+    this.searchFactory = searchFactory;
+    filter = resolveFilter(conf);
   }
 
   @Override
   public void Authenticate(String user, String password) throws AuthenticationException {
-
-    Hashtable<String, Object> env = new Hashtable<String, Object>();
-    env.put(Context.INITIAL_CONTEXT_FACTORY, "com.sun.jndi.ldap.LdapCtxFactory");
-    env.put(Context.PROVIDER_URL, ldapURL);
-
-    // If the domain is available in the config, then append it unless domain is
-    // already part of the username. LDAP providers like Active Directory use a
-    // fully qualified user name like foo@bar.com.
-    if (!hasDomain(user) && ldapDomain != null) {
-      user  = user + "@" + ldapDomain;
-    }
-
-    if (password == null || password.isEmpty() || password.getBytes()[0] == 0) {
-      throw new AuthenticationException("Error validating LDAP user:" +
-          " a null or blank password has been provided");
-    }
-
-    env.put(Context.SECURITY_AUTHENTICATION, "simple");
-    env.put(Context.SECURITY_CREDENTIALS, password);
-
-    // setup the security principal
-    String bindDN   = null;
-    DirContext ctx  = null;
-    String userDN   = null;
-    String userName = null;
-    Exception ex    = null;
-
-    if (!isDN(user) && !hasDomain(user) && userBases.size() > 0) {
-      ListIterator<String> listIter = userBases.listIterator();
-      while (listIter.hasNext()) {
-        try {
-          bindDN = listIter.next().replaceAll("%s", user);
-          env.put(Context.SECURITY_PRINCIPAL, bindDN);
-          LOG.debug("Connecting using DN " + bindDN + " at url " + ldapURL);
-          ctx = new InitialDirContext(env);
-          break;
-        } catch (NamingException e) {
-          ex = e;
-        }
-      }
-    } else {
-      env.put(Context.SECURITY_PRINCIPAL, user);
-      LOG.debug("Connecting using principal " + user + " at url " + ldapURL);
-      try {
-        ctx = new InitialDirContext(env);
-      } catch (NamingException e) {
-        ex = e;
-      }
-    }
-
-    if (ctx == null) {
-      LOG.debug("Could not connect to the LDAP Server:Authentication failed for " + user);
-      throw new AuthenticationException("LDAP Authentication failed for user", ex);
-    }
-
-    LOG.debug("Connected using principal=" + user + " at url=" + ldapURL);
+    DirSearch search = null;
     try {
-      if (isDN(user) || hasDomain(user)) {
-        userName = extractName(user);
-      } else {
-        userName = user;
-      }
-
-      // if a custom LDAP query is specified, it takes precedence over other configuration properties.
-      // if the user being authenticated is part of the resultset from the custom query, it succeeds.
-      if (customQuery != null) {
-        List<String> resultList = executeLDAPQuery(ctx, customQuery, baseDN);
-        if (resultList != null) {
-          for (String matchedDN : resultList) {
-            LOG.info("<queried user=" + matchedDN.split(",",2)[0].split("=",2)[1] + ",user=" + user + ">");
-            if (matchedDN.split(",",2)[0].split("=",2)[1].equalsIgnoreCase(user) ||
-                matchedDN.equalsIgnoreCase(user)) {
-              LOG.info("Authentication succeeded based on result set from LDAP query");
-              return;
-            }
-          }
-        }
-        LOG.info("Authentication failed based on result set from custom LDAP query");
-        throw new AuthenticationException("Authentication failed: LDAP query " +
-            "from property returned no data");
-      } else if (userBases.size() > 0) {
-        if (isDN(user)) {
-          userDN = findUserDNByDN(ctx, user);
-        } else {
-          if (userDN == null) {
-            userDN = findUserDNByPattern(ctx, userName);
-          }
-
-          if (userDN == null) {
-            userDN = findUserDNByName(ctx, userName);
-          }
-        }
-
-        // This should not be null because we were allowed to bind with this username
-        // safe check in case we were able to bind anonymously.
-        if (userDN == null) {
-          throw new AuthenticationException("Authentication failed: User search failed");
-        }
-
-        // This section checks if the user satisfies the specified user filter.
-        if (userFilter.size() > 0) {
-          LOG.info("Authenticating user " + user + " using user filter");
-
-          if (userDN != null) {
-            LOG.info("User filter partially satisfied");
-          }
-
-          boolean success = false;
-          for (String filteredUser : userFilter) {
-            if (filteredUser.equalsIgnoreCase(userName)) {
-              LOG.debug("User filter entirely satisfied");
-              success = true;
-              break;
-            }
-          }
-
-          if (!success) {
-            LOG.info("Authentication failed based on user membership");
-            throw new AuthenticationException("Authentication failed: User not a member " +
-                "of specified list");
-          }
-        }
-
-        // This section checks if the user satisfies the specified user filter.
-        if (groupFilter.size() > 0) {
-          LOG.debug("Authenticating user " + user + " using group membership");
-          List<String> userGroups = getGroupsForUser(ctx, userDN);
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("User member of :");
-            prettyPrint(userGroups);
-          }
-
-          if (userGroups != null) {
-            for (String elem : userGroups) {
-              String shortName = ((elem.split(","))[0].split("="))[1];
-              if (groupFilter.contains(shortName)) {
-                LOG.info("Authentication succeeded based on group membership");
-                return;
-              }
-            }
-          }
-
-          LOG.debug("Authentication failed: User is not a member of configured groups");
-          throw new AuthenticationException("Authentication failed: User not a member of " +
-              "listed groups");
-        }
-        LOG.info("Authentication succeeded using ldap user search");
-        return;
-      }
-      // Ideally we should not be here. Indicates partially configured LDAP Service.
-      // We allow it for now for backward compatibility.
-      LOG.info("Simple password authentication succeeded");
-    } catch (NamingException e) {
-      throw new AuthenticationException("LDAP Authentication failed for user", e);
+      search = createDirSearch(user, password);
+      applyFilter(search, user);
     } finally {
-      try {
-        if (ctx != null) {
-          ctx.close();
-        }
-      } catch(Exception e) {
-        LOG.warn("Exception when closing LDAP context:" + e.getMessage());
-      }
+      ServiceUtils.cleanup(LOG, search);
     }
   }
 
-  private boolean hasDomain(String userName) {
-    return (ServiceUtils.indexOfDomainMatch(userName) > 0);
-  }
-
-  private static void prettyPrint(List<String> list) {
-    for (String elem : list) {
-      LOG.debug("    " + elem);
+  private DirSearch createDirSearch(String user, String password) throws AuthenticationException {
+    if (StringUtils.isBlank(user)) {
+      throw new AuthenticationException("Error validating LDAP user:"
+          + " a null or blank user name has been provided");
     }
-  }
-
-  private static void prettyPrint(Attributes attrs) {
-    NamingEnumeration<? extends Attribute> set = attrs.getAll();
-    try {
-      NamingEnumeration<?> list = null;
-      while (set.hasMore()) {
-        Attribute attr = set.next();
-        list = attr.getAll();
-        String attrVals = "";
-        while (list.hasMore()) {
-          attrVals += list.next() + "+";
-        }
-        LOG.debug(attr.getID() + ":::" + attrVals);
-      }
-    } catch (Exception e) {
-      System.out.println("Error occurred when reading ldap data:" + e.getMessage());
+    if (StringUtils.isBlank(password) || password.getBytes()[0] == 0) {
+      throw new AuthenticationException("Error validating LDAP user:"
+          + " a null or blank password has been provided");
     }
-  }
-
-  /**
-   * This helper method attempts to find a DN given a unique groupname.
-   * Various LDAP implementations have different keys/properties that store this unique ID.
-   * So the first attempt is to find an entity with objectClass=group && CN=groupName
-   * @param ctx DirContext for the LDAP Connection.
-   * @param baseDN BaseDN for this LDAP directory where the search is to be performed.
-   * @param groupName A unique groupname that is to be located in the LDAP.
-   * @return LDAP DN if the group is found in LDAP, null otherwise.
-   */
-  public static String findGroupDNByName(DirContext ctx, String baseDN, String groupName)
-    throws NamingException {
-    String searchFilter  = "(&(objectClass=" + groupClass_attr + ")(" + guid_attr + "=" + groupName + "))";
-    List<String> results = null;
-
-    results = findDNByName(ctx, baseDN, searchFilter, 2);
-
-    if (results == null) {
-      return null;
-    } else if (results.size() > 1) {
-      //make sure there is not another item available, there should be only 1 match
-      LOG.info("Matched multiple groups for the group: " + groupName + ",returning null");
-      return null;
-    }
-    return results.get(0);
-  }
-
-  /**
-   * This helper method attempts to find an LDAP group entity given a unique name using a
-   * user-defined pattern for GROUPBASE.The list of group bases is defined by the user via property
-   * "hive.server2.authentication.ldap.groupDNPattern" in the hive-site.xml.
-   * Users can use %s where the actual groupname is to be substituted in the LDAP Query.
-   * @param ctx DirContext for the LDAP Connection.
-   * @param groupName A unique groupname that is to be located in the LDAP.
-   * @return LDAP DN of given group if found in the directory, null otherwise.
-   */
-  public static String findGroupDNByPattern(DirContext ctx, String groupName)
-      throws NamingException {
-    return findDNByPattern(ctx, groupName, groupBases);
-  }
-
-  public static String findDNByPattern(DirContext ctx, String name, List<String> nodes)
-      throws NamingException {
-    String searchFilter;
-    String searchBase;
-    SearchResult searchResult = null;
-    NamingEnumeration<SearchResult> results;
-
-    String[] returnAttributes     = new String[0]; // empty set
-    SearchControls searchControls = new SearchControls();
-
-    searchControls.setSearchScope(SearchControls.SUBTREE_SCOPE);
-    searchControls.setReturningAttributes(returnAttributes);
-
-    for (String node : nodes) {
-      searchFilter = "(" + (node.substring(0,node.indexOf(","))).replaceAll("%s", name) + ")";
-      searchBase   = node.split(",",2)[1];
-      results      = ctx.search(searchBase, searchFilter, searchControls);
-
-      if(results.hasMoreElements()) {
-        searchResult = results.nextElement();
-        //make sure there is not another item available, there should be only 1 match
-        if(results.hasMoreElements()) {
-          LOG.warn("Matched multiple entities for the name: " + name);
-          return null;
+    List<String> principals = LdapUtils.createCandidatePrincipals(conf, user);
+    for (Iterator<String> iterator = principals.iterator(); iterator.hasNext();) {
+      String principal = iterator.next();
+      try {
+        return searchFactory.getInstance(conf, principal, password);
+      } catch (AuthenticationException ex) {
+        if (!iterator.hasNext()) {
+          throw ex;
         }
-        return searchResult.getNameInNamespace();
       }
     }
-    return null;
+    throw new AuthenticationException(
+        String.format("No candidate principals for %s was found.", user));
   }
 
-  /**
-   * This helper method attempts to find a DN given a unique username.
-   * Various LDAP implementations have different keys/properties that store this unique userID.
-   * Active Directory has a "sAMAccountName" that appears reliable,openLDAP uses "uid"
-   * So the first attempt is to find an entity with objectClass=person||user where
-   * (uid||sAMAccountName) matches the given username.
-   * The second attempt is to use CN attribute for wild card matching and then match the
-   * username in the DN.
-   * @param ctx DirContext for the LDAP Connection.
-   * @param baseDN BaseDN for this LDAP directory where the search is to be performed.
-   * @param userName A unique userid that is to be located in the LDAP.
-   * @return LDAP DN if the user is found in LDAP, null otherwise.
-   */
-  public static String findUserDNByName(DirContext ctx, String userName)
-      throws NamingException {
-    if (userBases.size() == 0) {
-      return null;
-    }
-
-    String baseFilter    = "(&(|(objectClass=person)(objectClass=user)(objectClass=inetOrgPerson))";
-    String suffix[]      = new String[] {
-                             "(|(uid=" + userName + ")(sAMAccountName=" + userName + ")))",
-                             "(|(cn=*" + userName + "*)))"
-                           };
-
-    String searchFilter           = null;
-    List<String> results          = null;
-    ListIterator<String> listIter = userBases.listIterator();
-
-    for (int i = 0; i < suffix.length; i++) {
-      searchFilter = baseFilter + suffix[i];
-
-      while (listIter.hasNext()) {
-        results = findDNByName(ctx, listIter.next().split(",",2)[1], searchFilter, 2);
-
-        if(results == null) {
-          continue;
-        }
-
-        if(results != null && results.size() > 1) {
-          //make sure there is not another item available, there should be only 1 match
-          LOG.info("Matched multiple users for the user: " + userName + ",returning null");
-          return null;
-        }
-        return results.get(0);
+  private static Filter resolveFilter(HiveConf conf) {
+    for (FilterFactory filterProvider : FILTER_FACTORIES) {
+      Filter filter = filterProvider.getInstance(conf);
+      if (filter != null) {
+        return filter;
       }
     }
     return null;
   }
 
-  /**
-   * This helper method attempts to find a username given a DN.
-   * Various LDAP implementations have different keys/properties that store this unique userID.
-   * Active Directory has a "sAMAccountName" that appears reliable,openLDAP uses "uid"
-   * So the first attempt is to find an entity with objectClass=person||user where
-   * (uid||sAMAccountName) matches the given username.
-   * The second attempt is to use CN attribute for wild card matching and then match the
-   * username in the DN.
-   * @param ctx DirContext for the LDAP Connection.
-   * @param baseDN BaseDN for this LDAP directory where the search is to be performed.
-   * @param userName A unique userid that is to be located in the LDAP.
-   * @return LDAP DN if the user is found in LDAP, null otherwise.
-   */
-  public static String findUserDNByDN(DirContext ctx, String userDN)
-      throws NamingException {
-    if (!isDN(userDN)) {
-      return null;
-    }
-
-    String baseDN        = extractBaseDN(userDN);
-    List<String> results = null;
-    // we are using the first part of the userDN in the search criteria.
-    // We know the DN is legal as we are able to bind with it, this is to confirm that its a user.
-    String searchFilter  = "(&(|(objectClass=person)(objectClass=user)(objectClass=inetOrgPerson))("
-                             +  userDN.substring(0,userDN.indexOf(",")) + "))";
-
-    results = findDNByName(ctx, baseDN, searchFilter, 2);
-
-    if (results == null) {
-      return null;
-    }
-
-    if(results.size() > 1) {
-      //make sure there is not another item available, there should be only 1 match
-      LOG.info("Matched multiple users for the user: " + userDN + ",returning null");
-      return null;
-    }
-    return results.get(0);
-  }
-
-  public static List<String> findDNByName(DirContext ctx, String baseDN,
-      String searchString, int limit) throws NamingException {
-    SearchResult searchResult     = null;
-    List<String> retValues        = null;
-    String matchedDN              = null;
-    SearchControls searchControls = new SearchControls();
-    String[] returnAttributes     = new String[0]; //empty set
-
-    searchControls.setSearchScope(SearchControls.SUBTREE_SCOPE);
-    searchControls.setReturningAttributes(returnAttributes);
-    if (limit > 0) {
-      searchControls.setCountLimit(limit); // limit the result set to limit the size of resultset
-    }
-
-    NamingEnumeration<SearchResult> results = ctx.search(baseDN, searchString, searchControls);
-    while(results.hasMoreElements()) {
-      searchResult = results.nextElement();
-      matchedDN    = searchResult.getNameInNamespace();
-
-      if (retValues == null) {
-        retValues = new ArrayList<String>();
-      }
-      retValues.add(matchedDN);
-    }
-    return retValues;
-  }
-
-  /**
-   * This helper method attempts to find a UserDN given a unique username from a
-   * user-defined pattern for USERBASE. The list of user bases is defined by the user
-   * via property "hive.server2.authentication.ldap.userDNPattern" in the hive-site.xml.
-   * Users can use %s where the actual username is to be subsituted in the LDAP Query.
-   * @param ctx DirContext for the LDAP Connection.
-   * @param userName A unique userid that is to be located in the LDAP.
-   * @return LDAP DN of given user if found in the directory, null otherwise.
-   */
-  public static String findUserDNByPattern(DirContext ctx, String userName)
-      throws NamingException {
-    return findDNByPattern(ctx, userName, userBases);
-  }
-
-  /**
-   * This helper method finds all the groups a given user belongs to.
-   * This method relies on the attribute,configurable via HIVE_SERVER2_PLAIN_LDAP_GROUPMEMBERSHIP_KEY,
-   * being set on the user entry that references the group. The returned list ONLY includes direct
-   * groups the user belongs to. Parent groups of these direct groups are NOT included.
-   * @param ctx DirContext for the LDAP Connection.
-   * @param userDN A unique userDN that is to be located in the LDAP.
-   * @return List of Group DNs the user belongs to, emptylist otherwise.
-   */
-  public static List<String> getGroupsForUser(DirContext ctx, String userDN)
-      throws NamingException {
-    List<String> groupList        = new ArrayList<String>();
-    String user                   = extractName(userDN);
-    String searchFilter           = "(&(objectClass=" + groupClass_attr + ")(|(" +
-                                      groupMembership_attr + "=" + userDN + ")(" +
-                                      groupMembership_attr + "=" + user + ")))";
-    SearchControls searchControls = new SearchControls();
-    NamingEnumeration<SearchResult> results = null;
-    SearchResult result = null;
-    String groupBase = null;
-
-    LOG.debug("getGroupsForUser:searchFilter=" + searchFilter);
-    String[] attrIDs = new String[0];
-    searchControls.setSearchScope(SearchControls.SUBTREE_SCOPE);
-    searchControls.setReturningAttributes(attrIDs);
-
-    ListIterator<String> listIter = groupBases.listIterator();
-    while (listIter.hasNext()) {
-      try {
-        groupBase = listIter.next().split(",", 2)[1];
-        LOG.debug("Searching for groups under " + groupBase);
-        results   = ctx.search(groupBase, searchFilter, searchControls);
-
-        while(results.hasMoreElements()) {
-          result = results.nextElement();
-          LOG.debug("Found Group:" + result.getNameInNamespace());
-          groupList.add(result.getNameInNamespace());
-        }
-      } catch (NamingException e) {
-        LOG.warn("Exception searching for user groups", e);
-      }
-    }
-
-    return groupList;
-  }
-
-  /**
-   * This method helps execute a LDAP query defined by the user via property
-   * "hive.server2.authentication.ldap.customLDAPQuery"
-   * A full LDAP query that LDAP Atn provider uses to execute against LDAP Server.
-   * If this query return a null resultset, the LDAP Provider fails the authentication request.
-   * If the LDAP query returns a list of DNs, a check is performed to confirm one
-   * of the entries is for the user being authenticated.
-   * For example: (&(objectClass=group)(objectClass=top)(instanceType=4)(cn=Domain*))
-   * (&(objectClass=person)(|(sAMAccountName=admin)
-   *                       (|(memberOf=CN=Domain Admins,CN=Users,DC=domain,DC=com)
-   *                         (memberOf=CN=Administrators,CN=Builtin,DC=domain,DC=com))))
-   * @param ctx DirContext to execute this query within.
-   * @param query User-defined LDAP Query string to be used to authenticate users.
-   * @param rootDN BaseDN at which to execute the LDAP query, typically rootDN for the LDAP.
-   * @return List of LDAP DNs returned from executing the LDAP Query.
-   */
-  public static List<String> executeLDAPQuery(DirContext ctx, String query, String rootDN)
-      throws NamingException {
-    if (rootDN == null) {
-      return null;
-    }
-
-    SearchControls searchControls = new SearchControls();
-    List<String> list             = new ArrayList<String>();
-    String[] returnAttributes;
-    if (groupMembership_attr != null) {
-      // retrieve the attributes that are meant to desginate user DNs
-      returnAttributes = new String[] { groupMembership_attr };
-    } else {
-      returnAttributes = new String[0]; //empty set
-    }
-
-    searchControls.setSearchScope(SearchControls.SUBTREE_SCOPE);
-    searchControls.setReturningAttributes(returnAttributes);
-
-    LOG.info("Using a user specified LDAP query for adjudication:" + query + ",baseDN=" + rootDN);
-    NamingEnumeration<SearchResult> results = ctx.search(rootDN, query, searchControls);
-    SearchResult searchResult = null;
-    while(results.hasMoreElements()) {
-      searchResult = results.nextElement();
-      if (groupMembership_attr != null) {
-        Attribute userAttribute = searchResult.getAttributes().get(groupMembership_attr);
-        if (userAttribute != null) {
-          list.add((String)userAttribute.get());
-          continue;
-        }
+  private void applyFilter(DirSearch client, String user) throws AuthenticationException {
+    if (filter != null) {
+      if (LdapUtils.hasDomain(user)) {
+        filter.apply(client, LdapUtils.extractUserName(user));
+      } else {
+        filter.apply(client, user);
       }
-
-      list.add(searchResult.getNameInNamespace());
-      LOG.debug("LDAPAtn:executeLDAPQuery()::Return set size " + list.get(list.size() - 1));
     }
-    return list;
-  }
-
-  public static boolean isDN(String name) {
-    return (name.indexOf("=") > -1);
-  }
-
-  public static String extractName(String dn) {
-    int domainIdx = ServiceUtils.indexOfDomainMatch(dn);
-    if (domainIdx > 0) {
-      return dn.substring(0, domainIdx);
-    }
-
-    if (dn.indexOf("=") > -1) {
-      return dn.substring(dn.indexOf("=") + 1, dn.indexOf(","));
-    }
-    return dn;
-  }
-
-  public static String extractBaseDN(String dn) {
-    if (dn.indexOf(",") > -1) {
-      return dn.substring(dn.indexOf(",") + 1);
-    }
-    return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/ChainFilterFactory.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/ChainFilterFactory.java b/service/src/java/org/apache/hive/service/auth/ldap/ChainFilterFactory.java
new file mode 100644
index 0000000..e6255e8
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/ChainFilterFactory.java
@@ -0,0 +1,78 @@
+/**
+ * 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.hive.service.auth.ldap;
+
+import com.google.common.collect.ImmutableList;
+import java.util.ArrayList;
+import java.util.List;
+import javax.security.sasl.AuthenticationException;
+import org.apache.hadoop.hive.conf.HiveConf;
+
+/**
+ * A factory that produces a {@link Filter} that is implemented as a chain of other filters.
+ * The chain of filters are created as a result of
+ * {@link #getInstance(org.apache.hadoop.hive.conf.HiveConf) }
+ * method call. The resulting object filters out all users that don't pass <b>all</b>
+ * chained filters. The filters will be applied in the order they are mentioned in the factory
+ * constructor.
+ */
+public class ChainFilterFactory  implements FilterFactory {
+
+  private final List<FilterFactory> chainedFactories;
+
+  /**
+   * Constructs a factory for a chain of filters.
+   *
+   * @param factories The array of factories that will be used to construct a chain of filters.
+   */
+  public ChainFilterFactory(FilterFactory... factories) {
+    this.chainedFactories = ImmutableList.copyOf(factories);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public Filter getInstance(HiveConf conf) {
+    List<Filter> filters = new ArrayList<>();
+    for (FilterFactory factory : chainedFactories) {
+      Filter filter = factory.getInstance(conf);
+      if (filter != null) {
+        filters.add(filter);
+      }
+    }
+
+    return filters.isEmpty() ? null : new ChainFilter(ImmutableList.copyOf(filters));
+  }
+
+  private static final class ChainFilter implements Filter {
+
+    private final List<Filter> chainedFilters;
+
+    public ChainFilter(List<Filter> chainedFilters) {
+      this.chainedFilters = chainedFilters;
+    }
+
+    @Override
+    public void apply(DirSearch client, String user) throws AuthenticationException {
+      for (Filter filter : chainedFilters) {
+        filter.apply(client, user);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/CustomQueryFilterFactory.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/CustomQueryFilterFactory.java b/service/src/java/org/apache/hive/service/auth/ldap/CustomQueryFilterFactory.java
new file mode 100644
index 0000000..a0708c3
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/CustomQueryFilterFactory.java
@@ -0,0 +1,84 @@
+/**
+ * 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.hive.service.auth.ldap;
+
+import com.google.common.base.Strings;
+import java.util.List;
+import javax.naming.NamingException;
+import javax.security.sasl.AuthenticationException;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A factory for a {@link Filter} based on a custom query.
+ * <br>
+ * The produced filter object filters out all users that are not found in the search result
+ * of the query provided in Hive configuration.
+ * @see HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_CUSTOMLDAPQUERY
+ */
+public class CustomQueryFilterFactory implements FilterFactory {
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public Filter getInstance(HiveConf conf) {
+    String customQuery = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_CUSTOMLDAPQUERY);
+
+    if (Strings.isNullOrEmpty(customQuery)) {
+      return null;
+    }
+
+    return new CustomQueryFilter(customQuery);
+  }
+
+  private static final class CustomQueryFilter implements Filter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CustomQueryFilter.class);
+
+    private final String query;
+
+    public CustomQueryFilter(String query) {
+      this.query = query;
+    }
+
+    @Override
+    public void apply(DirSearch client, String user) throws AuthenticationException {
+      List<String> resultList;
+      try {
+        resultList = client.executeCustomQuery(query);
+      } catch (NamingException e) {
+        throw new AuthenticationException("LDAP Authentication failed for user", e);
+      }
+      if (resultList != null) {
+        for (String matchedDn : resultList) {
+          String shortUserName = LdapUtils.getShortName(matchedDn);
+          LOG.info("<queried user=" + shortUserName + ",user=" + user + ">");
+          if (shortUserName.equalsIgnoreCase(user) || matchedDn.equalsIgnoreCase(user)) {
+            LOG.info("Authentication succeeded based on result set from LDAP query");
+            return;
+          }
+        }
+      }
+      LOG.info("Authentication failed based on result set from custom LDAP query");
+      throw new AuthenticationException("Authentication failed: LDAP query "
+          + "from property returned no data");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/DirSearch.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/DirSearch.java b/service/src/java/org/apache/hive/service/auth/ldap/DirSearch.java
new file mode 100644
index 0000000..33b6088
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/DirSearch.java
@@ -0,0 +1,52 @@
+/**
+ * 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.hive.service.auth.ldap;
+
+import java.io.Closeable;
+import java.util.List;
+import javax.naming.NamingException;
+
+/**
+ * The object used for executing queries on the Directory Service.
+ */
+public interface DirSearch extends Closeable {
+
+  /**
+   * Finds user's distinguished name.
+   * @param user username
+   * @return DN for the specified username
+   * @throws NamingException
+   */
+  String findUserDn(String user) throws NamingException;
+
+  /**
+   * Finds groups that contain the specified user.
+   * @param userDn user's distinguished name
+   * @return list of groups
+   * @throws NamingException
+   */
+  List<String> findGroupsForUser(String userDn) throws NamingException;
+
+  /**
+   * Executes an arbitrary query.
+   * @param query any query
+   * @return list of names in the namespace
+   * @throws NamingException
+   */
+  List<String> executeCustomQuery(String query) throws NamingException;
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/DirSearchFactory.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/DirSearchFactory.java b/service/src/java/org/apache/hive/service/auth/ldap/DirSearchFactory.java
new file mode 100644
index 0000000..e6d9ff9
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/DirSearchFactory.java
@@ -0,0 +1,37 @@
+/**
+ * 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.hive.service.auth.ldap;
+
+import javax.security.sasl.AuthenticationException;
+import org.apache.hadoop.hive.conf.HiveConf;
+
+/**
+ * A factory for {@code DirSearch}.
+ */
+public interface DirSearchFactory {
+
+  /**
+   * Returns an instance of {@code DirSearch}.
+   * @param conf Hive configuration
+   * @param user username
+   * @param password user password
+   * @return instance of {@code DirSearch}
+   * @throws AuthenticationException
+   */
+  DirSearch getInstance(HiveConf conf, String user, String password) throws AuthenticationException;
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/Filter.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/Filter.java b/service/src/java/org/apache/hive/service/auth/ldap/Filter.java
new file mode 100644
index 0000000..fa72ced
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/Filter.java
@@ -0,0 +1,36 @@
+/**
+ * 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.hive.service.auth.ldap;
+
+import javax.security.sasl.AuthenticationException;
+
+/**
+ * The object that filters LDAP users.
+ * <br>
+ * The assumption is that this user was already authenticated by a previous bind operation.
+ */
+public interface Filter {
+
+  /**
+   * Applies this filter to the authenticated user.
+   * @param client LDAP client that will be used for execution of LDAP queries.
+   * @param user username
+   * @throws AuthenticationException
+   */
+  void apply(DirSearch client, String user) throws AuthenticationException;
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/FilterFactory.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/FilterFactory.java b/service/src/java/org/apache/hive/service/auth/ldap/FilterFactory.java
new file mode 100644
index 0000000..0708ccd
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/FilterFactory.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.hive.service.auth.ldap;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+
+/**
+ * Factory for the filter.
+ */
+public interface FilterFactory {
+
+  /**
+   * Returns an instance of the corresponding filter.
+   * @param conf Hive properties used to configure the filter.
+   * @return the filter or {@code null} if this filter doesn't support provided set of properties
+   */
+  Filter getInstance(HiveConf conf);
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/GroupFilterFactory.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/GroupFilterFactory.java b/service/src/java/org/apache/hive/service/auth/ldap/GroupFilterFactory.java
new file mode 100644
index 0000000..152c4b2
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/GroupFilterFactory.java
@@ -0,0 +1,90 @@
+/**
+ * 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.hive.service.auth.ldap;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import javax.naming.NamingException;
+import javax.security.sasl.AuthenticationException;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A factory for a {@link Filter} based on a list of allowed groups.
+ * <br>
+ * The produced filter object filters out all users that are not members of at least one of
+ * the groups provided in Hive configuration.
+ * @see HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPFILTER
+ */
+public final class GroupFilterFactory implements FilterFactory {
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public Filter getInstance(HiveConf conf) {
+    Collection<String> groupFilter = conf.getStringCollection(
+        HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPFILTER.varname);
+
+    if (groupFilter.isEmpty()) {
+      return null;
+    }
+
+    return new GroupFilter(groupFilter);
+  }
+
+  private static final class GroupFilter implements Filter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(GroupFilter.class);
+
+    private final Set<String> groupFilter = new HashSet<>();
+
+    GroupFilter(Collection<String> groupFilter) {
+      this.groupFilter.addAll(groupFilter);
+    }
+
+    @Override
+    public void apply(DirSearch ldap, String user) throws AuthenticationException {
+      LOG.info("Authenticating user '{}' using group membership", user);
+
+      List<String> memberOf = null;
+
+      try {
+        String userDn = ldap.findUserDn(user);
+        memberOf = ldap.findGroupsForUser(userDn);
+        LOG.debug("User {} member of : {}", userDn, memberOf);
+      } catch (NamingException e) {
+        throw new AuthenticationException("LDAP Authentication failed for user", e);
+      }
+
+      for (String groupDn : memberOf) {
+        String shortName = LdapUtils.getShortName(groupDn);
+        if (groupFilter.contains(shortName)) {
+          LOG.info("Authentication succeeded based on group membership");
+          return;
+        }
+      }
+      LOG.info("Authentication failed based on user membership");
+      throw new AuthenticationException("Authentication failed: "
+          + "User not a member of specified list");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/LdapSearch.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/LdapSearch.java b/service/src/java/org/apache/hive/service/auth/ldap/LdapSearch.java
new file mode 100644
index 0000000..65076ea
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/LdapSearch.java
@@ -0,0 +1,155 @@
+/**
+ * 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.hive.service.auth.ldap;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import javax.naming.NamingEnumeration;
+import javax.naming.NamingException;
+import javax.naming.directory.DirContext;
+import javax.naming.directory.SearchResult;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implements search for LDAP.
+ */
+public final class LdapSearch implements DirSearch {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LdapSearch.class);
+
+  private final String baseDn;
+  private final List<String> groupBases;
+  private final List<String> userBases;
+  private final List<String> userPatterns;
+
+  private final QueryFactory queries;
+
+  private final DirContext ctx;
+
+  /**
+   * Construct an instance of {@code LdapSearch}.
+   * @param conf Hive configuration
+   * @param ctx Directory service that will be used for the queries.
+   * @throws NamingException
+   */
+  public LdapSearch(HiveConf conf, DirContext ctx) throws NamingException {
+    baseDn = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_BASEDN);
+    userPatterns = LdapUtils.parseDnPatterns(conf,
+        HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERDNPATTERN);
+    groupBases = LdapUtils.patternsToBaseDns(LdapUtils.parseDnPatterns(conf,
+        HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPDNPATTERN));
+    userBases = LdapUtils.patternsToBaseDns(userPatterns);
+    this.ctx = ctx;
+    queries = new QueryFactory(conf);
+  }
+
+  /**
+   * Closes this search object and releases any system resources associated
+   * with it. If the search object is already closed then invoking this
+   * method has no effect.
+   */
+  @Override
+  public void close() {
+    try {
+      ctx.close();
+    } catch (NamingException e) {
+      LOG.warn("Exception when closing LDAP context:", e);
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public String findUserDn(String user) throws NamingException {
+    List<String> allLdapNames;
+    if (LdapUtils.isDn(user)) {
+      String userBaseDn = LdapUtils.extractBaseDn(user);
+      String userRdn = LdapUtils.extractFirstRdn(user);
+      allLdapNames = execute(Collections.singletonList(userBaseDn),
+          queries.findUserDnByRdn(userRdn)).getAllLdapNames();
+    } else {
+      allLdapNames = findDnByPattern(userPatterns, user);
+      if (allLdapNames.isEmpty()) {
+        allLdapNames = execute(userBases, queries.findUserDnByName(user)).getAllLdapNames();
+      }
+    }
+
+    if (allLdapNames.size() == 1) {
+      return allLdapNames.get(0);
+    } else {
+      LOG.info("Expected exactly one user result for the user: {}, but got {}. Returning null",
+          user, allLdapNames.size());
+      LOG.debug("Matched users: {}", allLdapNames);
+      return null;
+    }
+  }
+
+  private List<String> findDnByPattern(List<String> patterns, String name) throws NamingException {
+    for (String pattern : patterns) {
+      String baseDnFromPattern = LdapUtils.extractBaseDn(pattern);
+      String rdn = LdapUtils.extractFirstRdn(pattern).replaceAll("%s", name);
+      List<String> list = execute(Collections.singletonList(baseDnFromPattern),
+          queries.findDnByPattern(rdn)).getAllLdapNames();
+      if (!list.isEmpty()) {
+        return list;
+      }
+    }
+    return Collections.emptyList();
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public List<String> findGroupsForUser(String userDn) throws NamingException {
+    String userName = LdapUtils.extractUserName(userDn);
+    return execute(groupBases, queries.findGroupsForUser(userName, userDn)).getAllLdapNames();
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public List<String> executeCustomQuery(String query) throws NamingException {
+    return execute(Collections.singletonList(baseDn), queries.customQuery(query))
+        .getAllLdapNamesAndAttributes();
+  }
+
+  private SearchResultHandler execute(Collection<String> baseDns, Query query) {
+    List<NamingEnumeration<SearchResult>> searchResults = new ArrayList<>();
+    LOG.debug("Executing a query: '{}' with base DNs {}.", query.getFilter(), baseDns);
+    for (String aBaseDn : baseDns) {
+      try {
+        NamingEnumeration<SearchResult> searchResult = ctx.search(aBaseDn, query.getFilter(),
+            query.getControls());
+        if (searchResult != null) {
+          searchResults.add(searchResult);
+        }
+      } catch (NamingException ex) {
+        LOG.debug("Exception happened for query '" + query.getFilter() +
+            "' with base DN '" + aBaseDn + "'", ex);
+      }
+    }
+    return new SearchResultHandler(searchResults);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/LdapSearchFactory.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/LdapSearchFactory.java b/service/src/java/org/apache/hive/service/auth/ldap/LdapSearchFactory.java
new file mode 100644
index 0000000..71c3bfe
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/LdapSearchFactory.java
@@ -0,0 +1,64 @@
+/**
+ * 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.hive.service.auth.ldap;
+
+import java.util.Hashtable;
+import javax.naming.Context;
+import javax.naming.NamingException;
+import javax.naming.directory.DirContext;
+import javax.naming.directory.InitialDirContext;
+import javax.security.sasl.AuthenticationException;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A factory for LDAP search objects.
+ */
+public final class LdapSearchFactory implements DirSearchFactory {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LdapSearchFactory.class);
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public DirSearch getInstance(HiveConf conf, String principal, String password)
+      throws AuthenticationException {
+    try {
+      DirContext ctx = createDirContext(conf, principal, password);
+      return new LdapSearch(conf, ctx);
+    } catch (NamingException e) {
+      LOG.debug("Could not connect to the LDAP Server:Authentication failed for {}", principal);
+      throw new AuthenticationException("Error validating LDAP user", e);
+    }
+  }
+
+  private static DirContext createDirContext(HiveConf conf, String principal, String password)
+      throws NamingException {
+    Hashtable<String, Object> env = new Hashtable<String, Object>();
+    String ldapUrl = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_URL);
+    env.put(Context.INITIAL_CONTEXT_FACTORY, "com.sun.jndi.ldap.LdapCtxFactory");
+    env.put(Context.PROVIDER_URL, ldapUrl);
+    env.put(Context.SECURITY_AUTHENTICATION, "simple");
+    env.put(Context.SECURITY_CREDENTIALS, password);
+    env.put(Context.SECURITY_PRINCIPAL, principal);
+    LOG.debug("Connecting using principal {} to ldap url {}", principal, ldapUrl);
+    return new InitialDirContext(env);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/LdapUtils.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/LdapUtils.java b/service/src/java/org/apache/hive/service/auth/ldap/LdapUtils.java
new file mode 100644
index 0000000..df2ba6b
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/LdapUtils.java
@@ -0,0 +1,228 @@
+/**
+ * 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.hive.service.auth.ldap;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hive.service.ServiceUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Static utility methods related to LDAP authentication module.
+ */
+public final class LdapUtils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LdapUtils.class);
+
+  /**
+   * Extracts a base DN from the provided distinguished name.
+   * <br>
+   * <b>Example:</b>
+   * <br>
+   * "ou=CORP,dc=mycompany,dc=com" is the base DN for "cn=user1,ou=CORP,dc=mycompany,dc=com"
+   *
+   * @param dn distinguished name
+   * @return base DN
+   */
+  public static String extractBaseDn(String dn) {
+    final int indexOfFirstDelimiter = dn.indexOf(",");
+    if (indexOfFirstDelimiter > -1) {
+      return dn.substring(indexOfFirstDelimiter + 1);
+    }
+    return null;
+  }
+
+  /**
+   * Extracts the first Relative Distinguished Name (RDN).
+   * <br>
+   * <b>Example:</b>
+   * <br>
+   * For DN "cn=user1,ou=CORP,dc=mycompany,dc=com" this method will return "cn=user1"
+   * @param dn distinguished name
+   * @return first RDN
+   */
+  public static String extractFirstRdn(String dn) {
+    return dn.substring(0, dn.indexOf(","));
+  }
+
+  /**
+   * Extracts username from user DN.
+   * <br>
+   * <b>Examples:</b>
+   * <pre>
+   * LdapUtils.extractUserName("UserName")                        = "UserName"
+   * LdapUtils.extractUserName("UserName@mycorp.com")             = "UserName"
+   * LdapUtils.extractUserName("cn=UserName,dc=mycompany,dc=com") = "UserName"
+   * </pre>
+   * @param userDn
+   * @return
+   */
+  public static String extractUserName(String userDn) {
+    if (!isDn(userDn) && !hasDomain(userDn)) {
+      return userDn;
+    }
+
+    int domainIdx = ServiceUtils.indexOfDomainMatch(userDn);
+    if (domainIdx > 0) {
+      return userDn.substring(0, domainIdx);
+    }
+
+    if (userDn.contains("=")) {
+      return userDn.substring(userDn.indexOf("=") + 1, userDn.indexOf(","));
+    }
+    return userDn;
+  }
+
+  /**
+   * Gets value part of the first attribute in the provided RDN.
+   * <br>
+   * <b>Example:</b>
+   * <br>
+   * For RDN "cn=user1,ou=CORP" this method will return "user1"
+   * @param rdn Relative Distinguished Name
+   * @return value part of the first attribute
+   */
+  public static String getShortName(String rdn) {
+    return ((rdn.split(","))[0].split("="))[1];
+  }
+
+  /**
+   * Check for a domain part in the provided username.
+   * <br>
+   * <b>Example:</b>
+   * <br>
+   * <pre>
+   * LdapUtils.hasDomain("user1@mycorp.com") = true
+   * LdapUtils.hasDomain("user1")            = false
+   * </pre>
+   * @param userName username
+   * @return true if {@code userName} contains {@code @<domain>} part
+   */
+  public static boolean hasDomain(String userName) {
+    return (ServiceUtils.indexOfDomainMatch(userName) > 0);
+  }
+
+  /**
+   * Detects DN names.
+   * <br>
+   * <b>Example:</b>
+   * <br>
+   * <pre>
+   * LdapUtils.isDn("cn=UserName,dc=mycompany,dc=com") = true
+   * LdapUtils.isDn("user1")                           = false
+   * </pre>
+   * @param name name to be checked
+   * @return true if the provided name is a distinguished name
+   */
+  public static boolean isDn(String name) {
+    return name.contains("=");
+  }
+
+  /**
+   * Reads and parses DN patterns from Hive configuration.
+   * <br>
+   * If no patterns are provided in the configuration, then the base DN will be used.
+   * @param conf Hive configuration
+   * @param var variable to be read
+   * @return a list of DN patterns
+   * @see HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_BASEDN
+   * @see HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GUIDKEY
+   * @see HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPDNPATTERN
+   * @see HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERDNPATTERN
+   */
+  public static List<String> parseDnPatterns(HiveConf conf, HiveConf.ConfVars var) {
+    String patternsString = conf.getVar(var);
+    List<String> result = new ArrayList<>();
+    if (StringUtils.isBlank(patternsString)) {
+      String defaultBaseDn = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_BASEDN);
+      String guidAttr = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GUIDKEY);
+      if (StringUtils.isNotBlank(defaultBaseDn)) {
+        result.add(guidAttr + "=%s," + defaultBaseDn);
+      }
+    } else {
+      String[] patterns = patternsString.split(":");
+      for (String pattern : patterns) {
+        if (pattern.contains(",") && pattern.contains("=")) {
+          result.add(pattern);
+        } else {
+          LOG.warn("Unexpected format for " + var + "..ignoring " + pattern);
+        }
+      }
+    }
+    return result;
+  }
+
+  private static String patternToBaseDn(String pattern) {
+    if (pattern.contains("=%s")) {
+      return pattern.split(",", 2)[1];
+    }
+    return pattern;
+  }
+
+  /**
+   * Converts a collection of Distinguished Name patterns to a collection of base DNs.
+   * @param patterns Distinguished Name patterns
+   * @return a list of base DNs
+   * @see HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPDNPATTERN
+   * @see HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERDNPATTERN
+   */
+  public static List<String> patternsToBaseDns(Collection<String> patterns) {
+    List<String> result = new ArrayList<>();
+    for (String pattern : patterns) {
+      result.add(patternToBaseDn(pattern));
+    }
+    return result;
+  }
+
+  /**
+   * Creates a list of principals to be used for user authentication.
+   * @param conf Hive configuration
+   * @param user username
+   * @return a list of user's principals
+   */
+  public static List<String> createCandidatePrincipals(HiveConf conf, String user) {
+    if (hasDomain(user) || isDn(user)) {
+      return Collections.singletonList(user);
+    }
+
+    String ldapDomain = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_DOMAIN);
+    if (StringUtils.isNotBlank(ldapDomain)) {
+      return Collections.singletonList(user + "@" + ldapDomain);
+    }
+
+    List<String> userPatterns = parseDnPatterns(conf,
+        HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERDNPATTERN);
+    if (userPatterns.isEmpty()) {
+      return Collections.singletonList(user);
+    }
+
+    List<String> candidatePrincipals = new ArrayList<>();
+    for (String userPattern : userPatterns) {
+      candidatePrincipals.add(userPattern.replaceAll("%s", user));
+    }
+    return candidatePrincipals;
+  }
+
+  private LdapUtils() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/Query.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/Query.java b/service/src/java/org/apache/hive/service/auth/ldap/Query.java
new file mode 100644
index 0000000..b8bf938
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/Query.java
@@ -0,0 +1,154 @@
+/**
+ * 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.hive.service.auth.ldap;
+
+import com.google.common.base.Preconditions;
+import java.util.ArrayList;
+import java.util.List;
+import javax.naming.directory.SearchControls;
+import org.stringtemplate.v4.ST;
+
+/**
+ * The object that encompasses all components of a Directory Service search query.
+ * <br>
+ * @see LdapSearch
+ */
+public final class Query {
+
+  private final String filter;
+  private final SearchControls controls;
+
+  /**
+   * Constructs an instance of Directory Service search query.
+   * @param filter search filter
+   * @param controls search controls
+   */
+  public Query(String filter, SearchControls controls) {
+    this.filter = filter;
+    this.controls = controls;
+  }
+
+  /**
+   * Returns search filter.
+   * @return search filter
+   */
+  public String getFilter() {
+    return filter;
+  }
+
+  /**
+   * Returns search controls.
+   * @return search controls
+   */
+  public SearchControls getControls() {
+    return controls;
+  }
+
+  /**
+   * Creates Query Builder.
+   * @return query builder.
+   */
+  public static QueryBuilder builder() {
+    return new QueryBuilder();
+  }
+
+  /**
+   * A builder of the {@link Query}.
+   */
+  public static final class QueryBuilder {
+
+    private ST filterTemplate;
+    private final SearchControls controls = new SearchControls();
+    private final List<String> returningAttributes = new ArrayList<>();
+
+    private QueryBuilder() {
+      controls.setSearchScope(SearchControls.SUBTREE_SCOPE);
+      controls.setReturningAttributes(new String[0]);
+    }
+
+    /**
+     * Sets search filter template.
+     * @param filterTemplate search filter template
+     * @return the current instance of the builder
+     */
+    public QueryBuilder filter(String filterTemplate) {
+      this.filterTemplate = new ST(filterTemplate);
+      return this;
+    }
+
+    /**
+     * Sets mapping between names in the search filter template and actual values.
+     * @param key marker in the search filter template.
+     * @param value actual value
+     * @return the current instance of the builder
+     */
+    public QueryBuilder map(String key, String value) {
+      filterTemplate.add(key, value);
+      return this;
+    }
+
+    /**
+     * Sets attribute that should be returned in results for the query.
+     * @param attributeName attribute name
+     * @return the current instance of the builder
+     */
+    public QueryBuilder returnAttribute(String attributeName) {
+      returningAttributes.add(attributeName);
+      return this;
+    }
+
+    /**
+     * Sets the maximum number of entries to be returned as a result of the search.
+     * <br>
+     * 0 indicates no limit: all entries will be returned.
+     * @param limit The maximum number of entries that will be returned.
+     * @return the current instance of the builder
+     */
+    public QueryBuilder limit(int limit) {
+      controls.setCountLimit(limit);
+      return this;
+    }
+
+    private void validate() {
+      Preconditions.checkArgument(filterTemplate != null,
+          "filter is required for LDAP search query");
+    }
+
+    private String createFilter() {
+      return filterTemplate.render();
+    }
+
+    private void updateControls() {
+      if (!returningAttributes.isEmpty()) {
+        controls.setReturningAttributes(returningAttributes
+            .toArray(new String[returningAttributes.size()]));
+      }
+    }
+
+    /**
+     * Builds an instance of {@link Query}.
+     * @return configured directory service query
+     */
+    public Query build() {
+      validate();
+      String filter = createFilter();
+      updateControls();
+      return new Query(filter, controls);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/QueryFactory.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/QueryFactory.java b/service/src/java/org/apache/hive/service/auth/ldap/QueryFactory.java
new file mode 100644
index 0000000..e9172d3
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/QueryFactory.java
@@ -0,0 +1,135 @@
+/**
+ * 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.hive.service.auth.ldap;
+
+import com.google.common.base.Strings;
+import org.apache.hadoop.hive.conf.HiveConf;
+
+/**
+ * A factory for common types of directory service search queries.
+ */
+public final class QueryFactory {
+
+  private final String guidAttr;
+  private final String groupClassAttr;
+  private final String groupMembershipAttr;
+
+  /**
+   * Constructs the factory based on provided Hive configuration.
+   * @param conf Hive configuration
+   */
+  public QueryFactory(HiveConf conf) {
+    guidAttr = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GUIDKEY);
+    groupClassAttr = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPCLASS_KEY);
+    groupMembershipAttr = conf.getVar(
+        HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPMEMBERSHIP_KEY);
+  }
+
+  /**
+   * Returns a query for finding Group DN based on group unique ID.
+   * @param groupId group unique identifier
+   * @return an instance of {@link Query}
+   */
+  public Query findGroupDnById(String groupId) {
+    return Query.builder()
+        .filter("(&(objectClass=<groupClassAttr>)(<guidAttr>=<groupID>))")
+        .map("guidAttr", guidAttr)
+        .map("groupClassAttr", groupClassAttr)
+        .map("groupID", groupId)
+        .limit(2)
+        .build();
+  }
+
+  /**
+   * Returns a query for finding user DN based on user RDN.
+   * @param userRdn user RDN
+   * @return an instance of {@link Query}
+   */
+  public Query findUserDnByRdn(String userRdn) {
+    return Query.builder()
+        .filter("(&(|(objectClass=person)(objectClass=user)(objectClass=inetOrgPerson))"
+            + "(<userRdn>))")
+        .limit(2)
+        .map("userRdn", userRdn)
+        .build();
+  }
+
+  /**
+   * Returns a query for finding user DN based on DN pattern.
+   * <br>
+   * Name of this method was derived from the original implementation of LDAP authentication.
+   * This method should be replaced by {@link QueryFactory#findUserDnByRdn(java.lang.String).
+   *
+   * @param rdn user RDN
+   * @return an instance of {@link Query}
+   */
+  public Query findDnByPattern(String rdn) {
+    return Query.builder()
+        .filter("(<rdn>)")
+        .map("rdn", rdn)
+        .limit(2)
+        .build();
+  }
+
+  /**
+   * Returns a query for finding user DN based on user unique name.
+   * @param userName user unique name (uid or sAMAccountName)
+   * @return an instance of {@link Query}
+   */
+  public Query findUserDnByName(String userName) {
+    return Query.builder()
+        .filter("(&(|(objectClass=person)(objectClass=user)(objectClass=inetOrgPerson))"
+            + "(|(uid=<userName>)(sAMAccountName=<userName>)))")
+        .map("userName", userName)
+        .limit(2)
+        .build();
+  }
+
+  /**
+   * Returns a query for finding groups to which the user belongs.
+   * @param userName username
+   * @param userDn user DN
+   * @return an instance of {@link Query}
+   */
+  public Query findGroupsForUser(String userName, String userDn) {
+    return Query.builder()
+        .filter("(&(objectClass=<groupClassAttr>)(|(<groupMembershipAttr>=<userDn>)"
+            + "(<groupMembershipAttr>=<userName>)))")
+        .map("groupClassAttr", groupClassAttr)
+        .map("groupMembershipAttr", groupMembershipAttr)
+        .map("userName", userName)
+        .map("userDn", userDn)
+        .build();
+  }
+
+  /**
+   * Returns a query object created for the custom filter.
+   * <br>
+   * This query is configured to return a group membership attribute as part of the search result.
+   * @param searchFilter custom search filter
+   * @return an instance of {@link Query}
+   */
+  public Query customQuery(String searchFilter) {
+    Query.QueryBuilder builder = Query.builder();
+    builder.filter(searchFilter);
+    if (!Strings.isNullOrEmpty(groupMembershipAttr)) {
+      builder.returnAttribute(groupMembershipAttr);
+    }
+    return builder.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/SearchResultHandler.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/SearchResultHandler.java b/service/src/java/org/apache/hive/service/auth/ldap/SearchResultHandler.java
new file mode 100644
index 0000000..1b391f8
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/SearchResultHandler.java
@@ -0,0 +1,163 @@
+/**
+ * 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.hive.service.auth.ldap;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import javax.naming.NamingEnumeration;
+import javax.naming.NamingException;
+import javax.naming.directory.Attribute;
+import javax.naming.directory.SearchResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The object that handles Directory Service search results.
+ * In most cases it converts search results into a list of names in the namespace.
+ */
+public final class SearchResultHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SearchResultHandler.class);
+
+  private final Collection<NamingEnumeration<SearchResult>> searchResults;
+
+  /**
+   * Constructs a search result handler object for the provided search results.
+   * @param searchResults directory service search results
+   */
+  public SearchResultHandler(Collection<NamingEnumeration<SearchResult>> searchResults) {
+    this.searchResults = searchResults;
+  }
+
+  /**
+   * Returns all entries from the search result.
+   * @return a list of names in the namespace
+   * @throws NamingException
+   */
+  public List<String> getAllLdapNames() throws NamingException {
+    final List<String> result = new ArrayList<>();
+    handle(new RecordProcessor() {
+      @Override
+      public boolean process(SearchResult record) throws NamingException {
+        result.add(record.getNameInNamespace());
+        return true;
+      }
+    });
+    return result;
+  }
+
+  /**
+   * Checks whether search result contains exactly one entry.
+   * @return true if the search result contains a single entry.
+   * @throws NamingException
+   */
+  public boolean hasSingleResult() throws NamingException {
+    List<String> allResults = getAllLdapNames();
+    return allResults != null && allResults.size() == 1;
+  }
+
+  /**
+   * Returns a single entry from the search result.
+   * Throws {@code NamingException} if the search result doesn't contain exactly one entry.
+   * @return name in the namespace
+   * @throws NamingException
+   */
+  public String getSingleLdapName() throws NamingException {
+    List<String> allLdapNames = getAllLdapNames();
+    if (allLdapNames.size() == 1) {
+      return allLdapNames.get(0);
+    }
+    throw new NamingException("Single result was expected");
+  }
+
+  /**
+   * Returns all entries and all attributes for these entries.
+   * @return a list that includes all entries and all attributes from these entries.
+   * @throws NamingException
+   */
+  public List<String> getAllLdapNamesAndAttributes() throws NamingException {
+    final List<String> result = new ArrayList<>();
+    handle(new RecordProcessor() {
+      @Override
+      public boolean process(SearchResult record) throws NamingException {
+        result.add(record.getNameInNamespace());
+        NamingEnumeration<? extends Attribute> allAttributes = record.getAttributes().getAll();
+        while(allAttributes.hasMore()) {
+          Attribute attribute = allAttributes.next();
+          addAllAttributeValuesToResult(attribute.getAll());
+        }
+        return true;
+      }
+
+      private void addAllAttributeValuesToResult(NamingEnumeration values) throws NamingException {
+        while(values.hasMore()) {
+          result.add(String.valueOf(values.next()));
+        }
+      }
+
+    });
+    return result;
+  }
+
+  /**
+   * Allows for custom processing of the search results.
+   * @param processor {@link RecordProcessor} implementation
+   * @throws NamingException
+   */
+  public void handle(RecordProcessor processor) throws NamingException {
+    try {
+      for (NamingEnumeration<SearchResult> searchResult : searchResults) {
+        while (searchResult.hasMore()) {
+          if (!processor.process(searchResult.next())) {
+            return;
+          }
+        }
+      }
+    } finally {
+      for (NamingEnumeration<SearchResult> searchResult : searchResults) {
+        try {
+          searchResult.close();
+        } catch (NamingException ex) {
+          LOG.warn("Failed to close LDAP search result", ex);
+        }
+      }
+    }
+  }
+
+  /**
+   * An interface used by {@link SearchResultHandler} for processing records of
+   * a {@link SearchResult} on a per-record basis.
+   * <br>
+   * Implementations of this interface perform the actual work of processing each record,
+   * but don't need to worry about exception handling, closing underlying data structures,
+   * and combining results from several search requests.
+   * {@see SearchResultHandler}
+   */
+  public interface RecordProcessor {
+
+    /**
+     * Implementations must implement this method to process each record in {@link SearchResult}.
+     * @param record the {@code SearchResult} to precess
+     * @return {@code true} to continue processing, {@code false} to stop iterating
+     * over search results
+     * @throws NamingException
+     */
+    boolean process(SearchResult record) throws NamingException;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/990927e3/service/src/java/org/apache/hive/service/auth/ldap/UserFilterFactory.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/ldap/UserFilterFactory.java b/service/src/java/org/apache/hive/service/auth/ldap/UserFilterFactory.java
new file mode 100644
index 0000000..c8a6c88
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/auth/ldap/UserFilterFactory.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.hive.service.auth.ldap;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import javax.security.sasl.AuthenticationException;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A factory for a {@link Filter} based on a list of allowed users.
+ * <br>
+ * The produced filter object filters out all users that are not on the provided in
+ * Hive configuration list.
+ * @see HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER
+ */
+public final class UserFilterFactory implements FilterFactory {
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public Filter getInstance(HiveConf conf) {
+    Collection<String> userFilter = conf.getStringCollection(
+        HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER.varname);
+
+    if (userFilter.isEmpty()) {
+      return null;
+    }
+
+    return new UserFilter(userFilter);
+  }
+
+  private static final class UserFilter implements Filter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(UserFilter.class);
+
+    private final Set<String> userFilter = new HashSet<>();
+
+    UserFilter(Collection<String> userFilter) {
+      for (String userFilterItem : userFilter) {
+        this.userFilter.add(userFilterItem.toLowerCase());
+      }
+    }
+
+    @Override
+    public void apply(DirSearch ldap, String user) throws AuthenticationException {
+      LOG.info("Authenticating user '{}' using user filter", user);
+      String userName = LdapUtils.extractUserName(user).toLowerCase();
+      if (!userFilter.contains(userName)) {
+        LOG.info("Authentication failed based on user membership");
+        throw new AuthenticationException("Authentication failed: "
+            + "User not a member of specified list");
+      }
+    }
+  }
+}