You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2020/04/01 03:05:06 UTC

[impala] 02/02: IMPALA-2563: Support LDAP search bind operations

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

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

commit 4e6780ebf1dfa90aea01b3e35d3dc9ceb100eaee
Author: Thomas Tauber-Marshall <tm...@cloudera.com>
AuthorDate: Wed Mar 25 13:43:22 2020 -0700

    IMPALA-2563: Support LDAP search bind operations
    
    This patch adds a number of new options for controlling LDAP
    by restricting authentication to particular users and/or members of
    particular groups:
    --ldap_group_filter: comma separated list of authorized groups
    --ldap_user_filter: comma separated list of authorized users
    
    There are also options to control how LDAP is searched when applying
    these filters:
    --ldap_group_dn_pattern
    --ldap_group_membership_key
    --ldap_group_membership_class
    
    These options were modelled on equivalent options in Hive, see:
    https://cwiki.apache.org/confluence/display/Hive/User+and+Group+Filter+Support+with+LDAP+Atn+Provider+in+HiveServer2
    https://github.com/apache/hive/tree/master/service/src/java/org/apache/hive/service/auth/ldap
    
    This patch also refactors LDAP related functionality into a utility
    class, both to make authentication.cc more manageable and to
    facilitate follow up work that will add LDAP authentication options
    for the webserver.
    
    Testing:
    - Added a FE custom cluster test that sets --ldap_group_filter and
      --ldap_user_filter and verifies expected behavior.
    
    Change-Id: I7502a96e9a3c16faa67c03ffac54df2bdebbca8c
    Reviewed-on: http://gerrit.cloudera.org:8080/15570
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/common/global-flags.cc                      |   1 +
 be/src/rpc/authentication.cc                       | 175 ++----------
 be/src/rpc/authentication.h                        |  10 +
 be/src/util/CMakeLists.txt                         |   1 +
 be/src/util/ldap-util.cc                           | 311 +++++++++++++++++++++
 be/src/util/ldap-util.h                            |  73 +++++
 .../impala/customcluster/LdapImpalaShellTest.java  |  62 +++-
 fe/src/test/resources/users.ldif                   |  28 +-
 8 files changed, 495 insertions(+), 166 deletions(-)

diff --git a/be/src/common/global-flags.cc b/be/src/common/global-flags.cc
index d894574..79084ff 100644
--- a/be/src/common/global-flags.cc
+++ b/be/src/common/global-flags.cc
@@ -372,6 +372,7 @@ REMOVED_FLAG(enable_partitioned_hash_join);
 REMOVED_FLAG(enable_phj_probe_side_filtering);
 REMOVED_FLAG(enable_rm);
 REMOVED_FLAG(kerberos_reinit_interval);
+REMOVED_FLAG(ldap_manual_config);
 REMOVED_FLAG(llama_addresses);
 REMOVED_FLAG(llama_callback_port);
 REMOVED_FLAG(llama_host);
diff --git a/be/src/rpc/authentication.cc b/be/src/rpc/authentication.cc
index d126c6e..fb95e73 100644
--- a/be/src/rpc/authentication.cc
+++ b/be/src/rpc/authentication.cc
@@ -52,6 +52,7 @@
 #include "util/coding-util.h"
 #include "util/debug-util.h"
 #include "util/error-util.h"
+#include "util/ldap-util.h"
 #include "util/network-util.h"
 #include "util/os-util.h"
 #include "util/promise.h"
@@ -64,7 +65,6 @@
 #include "common/names.h"
 
 using boost::algorithm::is_any_of;
-using boost::algorithm::replace_all;
 using boost::algorithm::split;
 using boost::algorithm::trim;
 using boost::mt19937;
@@ -92,26 +92,13 @@ DEFINE_string(sasl_path, "", "Colon separated list of paths to look for SASL "
     "security library plugins.");
 DEFINE_bool(enable_ldap_auth, false,
     "If true, use LDAP authentication for client connections");
-
-DEFINE_string(ldap_uri, "", "The URI of the LDAP server to authenticate users against");
-DEFINE_bool(ldap_tls, false, "If true, use the secure TLS protocol to connect to the LDAP"
-    " server");
 DEFINE_string(ldap_ca_certificate, "", "The full path to the certificate file used to"
     " authenticate the LDAP server's certificate for SSL / TLS connections.");
-DEFINE_bool(ldap_passwords_in_clear_ok, false, "If set, will allow LDAP passwords "
-    "to be sent in the clear (without TLS/SSL) over the network.  This option should not "
-    "be used in production environments" );
-DEFINE_bool(ldap_allow_anonymous_binds, false, "(Advanced) If true, LDAP authentication "
-    "with a blank password (an 'anonymous bind') is allowed by Impala.");
-DEFINE_bool(ldap_manual_config, false, "Obsolete; Ignored");
-DEFINE_string(ldap_domain, "", "If set, Impala will try to bind to LDAP with a name of "
-    "the form <userid>@<ldap_domain>");
-DEFINE_string(ldap_baseDN, "", "If set, Impala will try to bind to LDAP with a name of "
-    "the form uid=<userid>,<ldap_baseDN>");
-DEFINE_string(ldap_bind_pattern, "", "If set, Impala will try to bind to LDAP with a name"
-     " of <ldap_bind_pattern>, but where the string #UID is replaced by the user ID. Use"
-     " to control the bind name precisely; do not set --ldap_domain or --ldap_baseDN with"
-     " this option");
+DEFINE_string(ldap_user_filter, "", "Comma separated list of usernames. If specified, "
+    "users must be on this list for athentication to succeed.");
+DEFINE_string(ldap_group_filter, "", "Comma separated list of groups. If specified, "
+    "users must belong to one of these groups for authentication to succeed.");
+
 DEFINE_string(internal_principals_whitelist, "hdfs", "(Advanced) Comma-separated list of "
     " additional usernames authorized to access Impala's internal APIs. Defaults to "
     "'hdfs' which is the system user that in certain deployments must access "
@@ -144,10 +131,6 @@ static string APP_NAME;
 static const string KERBEROS_MECHANISM = "GSSAPI";
 static const string PLAIN_MECHANISM = "PLAIN";
 
-// Required prefixes for ldap URIs:
-static const string LDAP_URI_PREFIX = "ldap://";
-static const string LDAPS_URI_PREFIX = "ldaps://";
-
 // We implement an "auxprop" plugin for the Sasl layer in order to have a hook in which
 // to log messages about the start of authentication. This is that plugin's name.
 static const string IMPALA_AUXPROP_PLUGIN = "impala-auxprop";
@@ -194,91 +177,8 @@ static int SaslLogCallback(void* context, int level, const char* message) {
   return SASL_OK;
 }
 
-// This callback is only called when we're providing LDAP authentication. This "check
-// pass" callback is our hook to ask the real LDAP server if we're allowed to log in or
-// not. We can be thought of as a proxy for LDAP logins - the user gives their password
-// to us, and we pass it to the real LDAP server.
-//
-// Note that this method uses ldap_sasl_bind_s(), which does *not* provide any security
-// to the connection between Impala and the LDAP server. You must either set --ldap_tls,
-// or have a URI which has "ldaps://" as the scheme in order to get a secure connection.
-// Use --ldap_ca_certificate to specify the location of the certificate used to confirm
-// the authenticity of the LDAP server certificate.
-//
-// user: The username to authenticate
-// pass: The password to use
-// passlen: The length of pass
-// Return: true on success, false otherwise
-static bool LdapCheckPass(const char* user, const char* pass, unsigned passlen) {
-  if (passlen == 0 && !FLAGS_ldap_allow_anonymous_binds) {
-    // Disable anonymous binds.
-    return false;
-  }
-
-  LDAP* ld;
-  int rc = ldap_initialize(&ld, FLAGS_ldap_uri.c_str());
-  if (rc != LDAP_SUCCESS) {
-    LOG(WARNING) << "Could not initialize connection with LDAP server ("
-                 << FLAGS_ldap_uri << "). Error: " << ldap_err2string(rc);
-    return false;
-  }
-
-  // Force the LDAP version to 3 to make sure TLS is supported.
-  int ldap_ver = 3;
-  ldap_set_option(ld, LDAP_OPT_PROTOCOL_VERSION, &ldap_ver);
-
-  // If -ldap_tls is turned on, and the URI is ldap://, issue a STARTTLS operation.
-  // Note that we'll ignore -ldap_tls when using ldaps:// because we've already
-  // got a secure connection (and the LDAP server will reject the STARTTLS).
-  if (FLAGS_ldap_tls && (FLAGS_ldap_uri.find(LDAP_URI_PREFIX) == 0)) {
-    int tls_rc = ldap_start_tls_s(ld, NULL, NULL);
-    if (tls_rc != LDAP_SUCCESS) {
-      LOG(WARNING) << "Could not start TLS secure connection to LDAP server ("
-                   << FLAGS_ldap_uri << "). Error: " << ldap_err2string(tls_rc);
-      ldap_unbind_ext(ld, NULL, NULL);
-      return false;
-    }
-    VLOG(2) << "Started TLS connection with LDAP server: " << FLAGS_ldap_uri;
-  }
-
-  // Map the user string into an acceptable LDAP "DN" (distinguished name)
-  string user_str = user;
-  if (!FLAGS_ldap_domain.empty()) {
-    // Append @domain if there isn't already an @ in the user string.
-    if (user_str.find("@") == string::npos) {
-      user_str = Substitute("$0@$1", user_str, FLAGS_ldap_domain);
-    }
-  } else if (!FLAGS_ldap_baseDN.empty()) {
-    user_str = Substitute("uid=$0,$1", user_str, FLAGS_ldap_baseDN);
-  } else if (!FLAGS_ldap_bind_pattern.empty()) {
-    user_str = FLAGS_ldap_bind_pattern;
-    replace_all(user_str, "#UID", user);
-  }
-
-  // Map the password into a credentials structure
-  struct berval cred;
-  cred.bv_val = const_cast<char*>(pass);
-  cred.bv_len = passlen;
-
-  VLOG_QUERY << "Trying simple LDAP bind for: " << user_str;
-
-  rc = ldap_sasl_bind_s(ld, user_str.c_str(), LDAP_SASL_SIMPLE, &cred,
-      NULL, NULL, NULL);
-  // Free ld
-  ldap_unbind_ext(ld, NULL, NULL);
-  if (rc != LDAP_SUCCESS) {
-    LOG(WARNING) << "LDAP authentication failure for " << user_str
-                 << " : " << ldap_err2string(rc);
-    return false;
-  }
-
-  VLOG_QUERY << "LDAP bind successful";
-
-  return true;
-}
-
-// Wrapper around the function we use to check passwords with LDAP which converts the
-// return value to something appropriate for SASL.
+// Wrapper around the function we use to check passwords with LDAP which has the function
+// signature required to work with SASL.
 //
 // conn: The Sasl connection struct, which we ignore
 // context: Ignored; always NULL
@@ -289,7 +189,8 @@ static bool LdapCheckPass(const char* user, const char* pass, unsigned passlen)
 // Return: SASL_OK on success, SASL_FAIL otherwise
 int SaslLdapCheckPass(sasl_conn_t* conn, void* context, const char* user,
     const char* pass, unsigned passlen, struct propctx* propctx) {
-  return LdapCheckPass(user, pass, passlen) ? SASL_OK : SASL_FAIL;
+  return AuthManager::GetInstance()->GetLdap()->LdapCheckPass(user, pass, passlen) ?
+      SASL_OK : SASL_FAIL;
 }
 
 // Sasl wants a way to ask us about some options, this function provides
@@ -544,7 +445,8 @@ bool BasicAuth(ThriftServer::ConnectionContext* connection_context,
   }
   string username = decoded.substr(0, colon);
   string password = decoded.substr(colon + 1);
-  bool ret = LdapCheckPass(username.c_str(), password.c_str(), password.length());
+  bool ret = AuthManager::GetInstance()->GetLdap()->LdapCheckPass(
+      username.c_str(), password.c_str(), password.length());
   if (ret) {
     // Authenication was successful, so set the username on the connection.
     connection_context->username = username;
@@ -1155,60 +1057,21 @@ void NoAuthProvider::SetupConnectionContext(
       MakeNetworkAddress(socket->getPeerAddress(), socket->getPeerPort());
 }
 
+AuthManager::AuthManager() {}
+
+AuthManager::~AuthManager() {}
+
 Status AuthManager::Init() {
   ssl_socket_factory_.reset(new TSSLSocketFactory(TLSv1_0));
 
   bool use_ldap = false;
-  const string excl_msg = "--$0 and --$1 are mutually exclusive "
-      "and should not be set together";
 
   // Get all of the flag validation out of the way
   if (FLAGS_enable_ldap_auth) {
     use_ldap = true;
-
-    if (!FLAGS_ldap_domain.empty()) {
-      if (!FLAGS_ldap_baseDN.empty()) {
-        return Status(Substitute(excl_msg, "ldap_domain", "ldap_baseDN"));
-      }
-      if (!FLAGS_ldap_bind_pattern.empty()) {
-        return Status(Substitute(excl_msg, "ldap_domain", "ldap_bind_pattern"));
-      }
-    } else if (!FLAGS_ldap_baseDN.empty()) {
-      if (!FLAGS_ldap_bind_pattern.empty()) {
-        return Status(Substitute(excl_msg, "ldap_baseDN", "ldap_bind_pattern"));
-      }
-    }
-
-    if (FLAGS_ldap_uri.empty()) {
-      return Status("--ldap_uri must be supplied when --ldap_enable_auth is set");
-    }
-
-    if ((FLAGS_ldap_uri.find(LDAP_URI_PREFIX) != 0) &&
-        (FLAGS_ldap_uri.find(LDAPS_URI_PREFIX) != 0)) {
-      return Status(Substitute("--ldap_uri must start with either $0 or $1",
-              LDAP_URI_PREFIX, LDAPS_URI_PREFIX ));
-    }
-
-    LOG(INFO) << "Using LDAP authentication with server " << FLAGS_ldap_uri;
-
-    if (!FLAGS_ldap_tls && (FLAGS_ldap_uri.find(LDAPS_URI_PREFIX) != 0)) {
-      if (FLAGS_ldap_passwords_in_clear_ok) {
-        LOG(WARNING) << "LDAP authentication is being used, but without TLS. "
-                     << "ALL PASSWORDS WILL GO OVER THE NETWORK IN THE CLEAR.";
-      } else {
-        return Status("LDAP authentication specified, but without TLS. "
-                      "Passwords would go over the network in the clear. "
-                      "Enable TLS with --ldap_tls or use an ldaps:// URI. "
-                      "To override this is non-production environments, "
-                      "specify --ldap_passwords_in_clear_ok");
-      }
-    } else if (FLAGS_ldap_ca_certificate.empty()) {
-      LOG(WARNING) << "LDAP authentication is being used with TLS, but without "
-                   << "an --ldap_ca_certificate file, the identity of the LDAP "
-                   << "server cannot be verified.  Network communication (and "
-                   << "hence passwords) could be intercepted by a "
-                   << "man-in-the-middle attack";
-    }
+    RETURN_IF_ERROR(ImpalaLdap::ValidateFlags());
+    ldap_.reset(new ImpalaLdap());
+    RETURN_IF_ERROR(ldap_->Init(FLAGS_ldap_user_filter, FLAGS_ldap_group_filter));
   }
 
   if (FLAGS_principal.empty() && !FLAGS_be_principal.empty()) {
diff --git a/be/src/rpc/authentication.h b/be/src/rpc/authentication.h
index 7b65233..96d091b 100644
--- a/be/src/rpc/authentication.h
+++ b/be/src/rpc/authentication.h
@@ -36,6 +36,8 @@ using namespace ::apache::thrift::transport;
 
 namespace impala {
 
+class ImpalaLdap;
+
 /// System-wide authentication manager responsible for initialising authentication systems,
 /// including SSL, Sasl and Kerberos, and for providing auth-enabled Thrift structures to
 /// servers and clients.
@@ -46,6 +48,9 @@ class AuthManager {
  public:
   static AuthManager* GetInstance() { return AuthManager::auth_manager_; }
 
+  AuthManager();
+  ~AuthManager();
+
   /// Set up internal and external AuthProvider classes. This also initializes SSL (via
   /// the creation of ssl_socket_factory_).
   Status Init();
@@ -61,6 +66,8 @@ class AuthManager {
   /// connection this applies to would be backend <-> statestore.
   AuthProvider* GetInternalAuthProvider();
 
+  ImpalaLdap* GetLdap() { return ldap_.get(); }
+
  private:
   /// One-time kerberos-specific environment variable setup. Sets variables like
   /// KRB5CCNAME and friends so that command-line flags take effect in the C++ and
@@ -81,6 +88,9 @@ class AuthManager {
   /// initialized, this will be created regardless of whether or not SSL credentials are
   /// specified. This factory isn't otherwise used.
   boost::scoped_ptr<TSSLSocketFactory> ssl_socket_factory_;
+
+  /// Used to authenticate usernames and passwords to LDAP.
+  std::unique_ptr<ImpalaLdap> ldap_;
 };
 
 
diff --git a/be/src/util/CMakeLists.txt b/be/src/util/CMakeLists.txt
index f3dc289..5ffcd17 100644
--- a/be/src/util/CMakeLists.txt
+++ b/be/src/util/CMakeLists.txt
@@ -61,6 +61,7 @@ add_library(Util
   impalad-metrics.cc
   jni-util.cc
   json-util.cc
+  ldap-util.cc
   logging-support.cc
   mem-info.cc
   memory-metrics.cc
diff --git a/be/src/util/ldap-util.cc b/be/src/util/ldap-util.cc
new file mode 100644
index 0000000..d573207
--- /dev/null
+++ b/be/src/util/ldap-util.cc
@@ -0,0 +1,311 @@
+// 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.
+
+#include "util/ldap-util.h"
+
+#include <ldap.h>
+#include <boost/algorithm/string.hpp>
+#include <gflags/gflags.h>
+#include <gutil/strings/split.h>
+#include <gutil/strings/util.h>
+
+#include "common/logging.h"
+
+#include "common/names.h"
+
+DEFINE_string(ldap_uri, "", "The URI of the LDAP server to authenticate users against");
+DEFINE_bool(ldap_tls, false, "If true, use the secure TLS protocol to connect to the LDAP"
+    " server");
+
+DEFINE_bool(ldap_passwords_in_clear_ok, false, "If set, will allow LDAP passwords "
+    "to be sent in the clear (without TLS/SSL) over the network.  This option should not "
+    "be used in production environments" );
+DEFINE_bool(ldap_allow_anonymous_binds, false, "(Advanced) If true, LDAP authentication "
+    "with a blank password (an 'anonymous bind') is allowed by Impala.");
+
+DEFINE_string(ldap_domain, "", "If set, Impala will try to bind to LDAP with a name of "
+    "the form <userid>@<ldap_domain>");
+DEFINE_string(ldap_baseDN, "", "If set, Impala will try to bind to LDAP with a name of "
+    "the form uid=<userid>,<ldap_baseDN>");
+DEFINE_string(ldap_bind_pattern, "", "If set, Impala will try to bind to LDAP with a name"
+     " of <ldap_bind_pattern>, but where the string #UID is replaced by the user ID. Use"
+     " to control the bind name precisely; do not set --ldap_domain or --ldap_baseDN with"
+     " this option");
+
+DEFINE_string(ldap_group_dn_pattern, "", "Colon separated list of patterns for the "
+    "'distinguished name' used to search for groups in the directory. Each pattern may "
+    "contain a '%s' which will be substituted with each group name from "
+    "--ldap_group_filter when doing group searches.");
+DEFINE_string(ldap_group_membership_key, "member",
+    "The LDAP attribute on group entries that indicates its members.");
+DEFINE_string(ldap_group_class_key, "groupOfNames",
+    "The LDAP objectClass each of the groups in --ldap_group_filter implements in LDAP.");
+
+DECLARE_string(ldap_ca_certificate);
+
+using boost::algorithm::replace_all;
+using namespace strings;
+
+namespace impala {
+
+// Required prefixes for ldap URIs:
+static const string LDAP_URI_PREFIX = "ldap://";
+static const string LDAPS_URI_PREFIX = "ldaps://";
+
+Status ImpalaLdap::ValidateFlags() {
+  const string excl_msg = "--$0 and --$1 are mutually exclusive "
+    "and should not be set together";
+
+  if (!FLAGS_ldap_domain.empty()) {
+    if (!FLAGS_ldap_baseDN.empty()) {
+      return Status(Substitute(excl_msg, "ldap_domain", "ldap_baseDN"));
+    }
+    if (!FLAGS_ldap_bind_pattern.empty()) {
+      return Status(Substitute(excl_msg, "ldap_domain", "ldap_bind_pattern"));
+    }
+  } else if (!FLAGS_ldap_baseDN.empty()) {
+    if (!FLAGS_ldap_bind_pattern.empty()) {
+      return Status(Substitute(excl_msg, "ldap_baseDN", "ldap_bind_pattern"));
+    }
+  }
+
+  if (FLAGS_ldap_uri.empty()) {
+    return Status("--ldap_uri must be supplied when --ldap_enable_auth is set");
+  }
+
+  if ((FLAGS_ldap_uri.find(LDAP_URI_PREFIX) != 0)
+      && (FLAGS_ldap_uri.find(LDAPS_URI_PREFIX) != 0)) {
+    return Status(Substitute(
+        "--ldap_uri must start with either $0 or $1", LDAP_URI_PREFIX, LDAPS_URI_PREFIX));
+  }
+
+  LOG(INFO) << "Using LDAP authentication with server " << FLAGS_ldap_uri;
+
+  if (!FLAGS_ldap_tls && (FLAGS_ldap_uri.find(LDAPS_URI_PREFIX) != 0)) {
+    if (FLAGS_ldap_passwords_in_clear_ok) {
+      LOG(WARNING) << "LDAP authentication is being used, but without TLS. "
+                   << "ALL PASSWORDS WILL GO OVER THE NETWORK IN THE CLEAR.";
+    } else {
+      return Status("LDAP authentication specified, but without TLS. "
+          "Passwords would go over the network in the clear. "
+          "Enable TLS with --ldap_tls or use an ldaps:// URI. "
+          "To override this is non-production environments, "
+          "specify --ldap_passwords_in_clear_ok");
+    }
+  } else if (FLAGS_ldap_ca_certificate.empty()) {
+    LOG(WARNING) << "LDAP authentication is being used with TLS, but without "
+                 << "an --ldap_ca_certificate file, the identity of the LDAP "
+                 << "server cannot be verified.  Network communication (and "
+                 << "hence passwords) could be intercepted by a "
+                 << "man-in-the-middle attack";
+  }
+  return Status::OK();
+}
+
+Status ImpalaLdap::Init(const std::string& user_filter, const std::string& group_filter) {
+  if (!user_filter.empty()) {
+    user_filter_ = Split(user_filter, ",");
+  }
+
+  if (!group_filter.empty()) {
+    if (FLAGS_ldap_group_dn_pattern.empty()) {
+      return Status("In order to apply an LDAP group filter, --ldap_group_dn_pattern "
+                    "must be specified.");
+    }
+    group_filter_ = Split(group_filter, ",");
+    vector<string> group_dns = Split(FLAGS_ldap_group_dn_pattern, ":");
+
+    // Build the list of DNs to search for groups by iterating through the
+    // DN patterns and replacing the optional '%s' with each group name, if present.
+    for (const string& group_dn_pattern : group_dns) {
+      if (group_dn_pattern.find("%s") != std::string::npos) {
+        for (const string& group : group_filter_) {
+          group_filter_dns_.push_back(
+              StringReplace(group_dn_pattern, "%s", group, /* replace_all */ false));
+        }
+      } else {
+        group_filter_dns_.push_back(group_dn_pattern);
+      }
+    }
+  }
+
+  return Status::OK();
+}
+
+bool ImpalaLdap::LdapCheckPass(const char* user, const char* pass, unsigned passlen) {
+  if (passlen == 0 && !FLAGS_ldap_allow_anonymous_binds) {
+    // Disable anonymous binds.
+    return false;
+  }
+
+  LDAP* ld;
+  int rc = ldap_initialize(&ld, FLAGS_ldap_uri.c_str());
+  if (rc != LDAP_SUCCESS) {
+    LOG(WARNING) << "Could not initialize connection with LDAP server (" << FLAGS_ldap_uri
+                 << "). Error: " << ldap_err2string(rc);
+    return false;
+  }
+
+  // Force the LDAP version to 3 to make sure TLS is supported.
+  int ldap_ver = 3;
+  ldap_set_option(ld, LDAP_OPT_PROTOCOL_VERSION, &ldap_ver);
+
+  // If -ldap_tls is turned on, and the URI is ldap://, issue a STARTTLS operation.
+  // Note that we'll ignore -ldap_tls when using ldaps:// because we've already
+  // got a secure connection (and the LDAP server will reject the STARTTLS).
+  if (FLAGS_ldap_tls && (FLAGS_ldap_uri.find(LDAP_URI_PREFIX) == 0)) {
+    int tls_rc = ldap_start_tls_s(ld, nullptr, nullptr);
+    if (tls_rc != LDAP_SUCCESS) {
+      LOG(WARNING) << "Could not start TLS secure connection to LDAP server ("
+                   << FLAGS_ldap_uri << "). Error: " << ldap_err2string(tls_rc);
+      ldap_unbind_ext(ld, nullptr, nullptr);
+      return false;
+    }
+    VLOG(2) << "Started TLS connection with LDAP server: " << FLAGS_ldap_uri;
+  }
+
+  // Map the user string into an acceptable LDAP "DN" (distinguished name)
+  string user_str = user;
+  if (!FLAGS_ldap_domain.empty()) {
+    // Append @domain if there isn't already an @ in the user string.
+    if (user_str.find("@") == string::npos) {
+      user_str = Substitute("$0@$1", user_str, FLAGS_ldap_domain);
+    }
+  } else if (!FLAGS_ldap_baseDN.empty()) {
+    user_str = Substitute("uid=$0,$1", user_str, FLAGS_ldap_baseDN);
+  } else if (!FLAGS_ldap_bind_pattern.empty()) {
+    user_str = FLAGS_ldap_bind_pattern;
+    replace_all(user_str, "#UID", user);
+  }
+
+  // Map the password into a credentials structure
+  struct berval cred;
+  cred.bv_val = const_cast<char*>(pass);
+  cred.bv_len = passlen;
+
+  VLOG_QUERY << "Trying simple LDAP bind for: " << user_str;
+
+  rc = ldap_sasl_bind_s(
+      ld, user_str.c_str(), LDAP_SASL_SIMPLE, &cred, nullptr, nullptr, nullptr);
+  // Free ld
+  if (rc != LDAP_SUCCESS) {
+    LOG(WARNING) << "LDAP authentication failure for " << user_str << " : "
+                 << ldap_err2string(rc);
+    ldap_unbind_ext(ld, nullptr, nullptr);
+    return false;
+  }
+
+  VLOG_QUERY << "LDAP bind successful";
+
+  if (!user_filter_.empty() && user_filter_.count(user) != 1) {
+    LOG(WARNING) << "LDAP authentication failure for " << user_str << ". Bind was "
+                 << "successful but user is not in the authorized user list.";
+    ldap_unbind_ext(ld, nullptr, nullptr);
+    return false;
+  }
+
+  if (!group_filter_.empty()) {
+    if (!CheckGroupMembership(ld, user_str)) {
+      LOG(WARNING) << "LDAP authentication failure for " << user_str << ". Bind was "
+                   << "successful but user is not in any of the required groups.";
+      ldap_unbind_ext(ld, nullptr, nullptr);
+      return false;
+    }
+  }
+  ldap_unbind_ext(ld, nullptr, nullptr);
+
+  return true;
+}
+
+bool ImpalaLdap::CheckGroupMembership(LDAP* ld, const string& user_str) {
+  // Construct a filter that will search for LDAP entries that represent groups
+  // (determined by having the group class key) and that contain the user trying to
+  // authenticate (determined by having a membership entry matching the user).
+  string filter = Substitute("(&(objectClass=$0)($1=$2))", FLAGS_ldap_group_class_key,
+      FLAGS_ldap_group_membership_key, user_str);
+  VLOG(2) << "Searching for groups with filter: " << filter;
+
+  for (const string& group_dn : group_filter_dns_) {
+    LDAPMessage* result;
+    // Search through LDAP starting at a base of 'group_dn' and including the entire
+    // subtree below it while applying 'filter'. This should return a list of all group
+    // entries encountered in the search that have the given user as a member.
+    int rc = ldap_search_ext_s(ld, group_dn.c_str(), LDAP_SCOPE_SUBTREE, filter.c_str(),
+        nullptr, false, nullptr, nullptr, nullptr, LDAP_MAXINT, &result);
+    if (rc != LDAP_SUCCESS) {
+      LOG(WARNING) << "LDAP search failed for " << filter << " with DN=" << group_dn
+                   << ": " << ldap_err2string(rc);
+      ldap_msgfree(result);
+      continue;
+    }
+
+    for (LDAPMessage* msg = ldap_first_message(ld, result); msg != nullptr;
+         msg = ldap_next_message(ld, msg)) {
+      int msg_type = ldap_msgtype(msg);
+      switch (msg_type) {
+        case LDAP_RES_SEARCH_ENTRY:
+          char* dn;
+          if ((dn = ldap_get_dn(ld, msg)) != nullptr) {
+            string short_name = GetShortName(dn);
+            if (group_filter_.count(short_name) == 1) {
+              ldap_memfree(dn);
+              ldap_msgfree(result);
+              return true;
+            }
+            ldap_memfree(dn);
+          } else {
+            LOG(WARNING) << "LDAP search error for " << filter << " with DN=" << group_dn
+                         << ": Was not able to get DN from search result.";
+          }
+          break;
+        case LDAP_RES_SEARCH_REFERENCE: {
+          LOG(WARNING) << "LDAP search error for " << filter << " with DN=" << group_dn
+                       << ": Following of referrals not supported, ignoring.";
+          char** referrals;
+          int parse_rc = ldap_parse_reference(ld, msg, &referrals, nullptr, 0);
+          if (parse_rc != LDAP_SUCCESS) {
+            LOG(WARNING) << "Was unable to parse LDAP search reference result: "
+                         << ldap_err2string(parse_rc);
+            break;
+          }
+
+          if (referrals != nullptr) {
+            for (int i = 0; referrals[i] != nullptr; ++i) {
+              LOG(WARNING) << "Got search reference: " << referrals[i];
+            }
+            ber_memvfree((void**)referrals);
+          }
+          break;
+        }
+        case LDAP_RES_SEARCH_RESULT:
+          // Indicates the end of the messages in the result. Nothing to do.
+          break;
+      }
+    }
+    ldap_msgfree(result);
+  }
+
+  return false;
+}
+
+string ImpalaLdap::GetShortName(const string& rdn) {
+  vector<string> attributes = Split(rdn, delimiter::Limit(",", 1));
+  vector<string> value = Split(attributes[0], delimiter::Limit("=", 1));
+  return value[1];
+}
+
+} // namespace impala
diff --git a/be/src/util/ldap-util.h b/be/src/util/ldap-util.h
new file mode 100644
index 0000000..8a34905
--- /dev/null
+++ b/be/src/util/ldap-util.h
@@ -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.
+
+#pragma once
+
+#include <string>
+#include <unordered_set>
+#include <vector>
+
+#include "common/status.h"
+
+struct ldap;
+typedef struct ldap LDAP;
+
+namespace impala {
+
+class Status;
+
+// Utility class for checking usernames and passwords in LDAP.
+class ImpalaLdap {
+ public:
+  static Status ValidateFlags();
+
+  /// 'user_filter' and 'group_filter' are optional comma separated lists specifying what
+  /// users and groups are allowed to authenticate.
+  Status Init(
+      const std::string& user_filter, const std::string& group_filter) WARN_UNUSED_RESULT;
+
+  /// Attempts to authenticate to LDAP using the given username and password, applying the
+  /// user or group filters as approrpriate. 'passlen' is the length of the password.
+  /// Returns true if authentication is successful.
+  ///
+  /// Note that this method uses ldap_sasl_bind_s(), which does *not* provide any security
+  /// to the connection between Impala and the LDAP server. You must either set
+  /// --ldap_tls or have a URI which has "ldaps://" as the scheme in order to get a secure
+  /// connection. Use --ldap_ca_certificate to specify the location of the certificate
+  /// used to confirm the authenticity of the LDAP server certificate.
+  bool LdapCheckPass(
+      const char* username, const char* password, unsigned passlen) WARN_UNUSED_RESULT;
+
+ private:
+  /// If non-empty, only users in this set can successfully authenticate.
+  std::unordered_set<std::string> user_filter_;
+
+  /// If non-empty, only users who belong to groups in this set can successfully
+  /// authenticate.
+  std::unordered_set<std::string> group_filter_;
+  /// The base DNs to use when preforming group searches.
+  std::vector<std::string> group_filter_dns_;
+
+  /// Searches LDAP to determine if 'user_str' belong to one of the groups in
+  /// 'group_filter_'. Returns true if so.
+  bool CheckGroupMembership(LDAP* ld, const std::string& user_str);
+
+  /// Returns the value part of the first attribute in the provided relative DN.
+  static std::string GetShortName(const std::string& rdn);
+};
+
+} // namespace impala
diff --git a/fe/src/test/java/org/apache/impala/customcluster/LdapImpalaShellTest.java b/fe/src/test/java/org/apache/impala/customcluster/LdapImpalaShellTest.java
index 777bf844..070aa33 100644
--- a/fe/src/test/java/org/apache/impala/customcluster/LdapImpalaShellTest.java
+++ b/fe/src/test/java/org/apache/impala/customcluster/LdapImpalaShellTest.java
@@ -32,7 +32,7 @@ import org.apache.directory.server.annotations.CreateTransport;
 import org.apache.directory.server.core.annotations.ApplyLdifFiles;
 import org.apache.directory.server.core.integ.CreateLdapServerRule;
 import org.apache.impala.testutil.ImpalaJdbcClient;
-import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Assume;
 import org.junit.Before;
 import org.junit.ClassRule;
@@ -58,26 +58,31 @@ public class LdapImpalaShellTest {
   private static final String testPassword_ = "12345";
   private static final String testUser2_ = "Test2Ldap";
   private static final String testPassword2_ = "abcde";
+  private static final String testUser3_ = "Test3Ldap";
+  private static final String testPassword3_ = "67890";
+  private static final String testUser4_ = "Test4Ldap";
+  private static final String testPassword4_ = "fghij";
+  // testUser_ and testUser2_ are members of this group.
+  private static final String testUserGroup_ = "group1";
 
   // The cluster will be set up to allow testUser_ to act as a proxy for delegateUser_.
   // Includes a special character to test HTTP path encoding.
   private static final String delegateUser_ = "proxyUser$";
 
-  @Before
-  public void setUp() throws Exception {
+  public void setUp(String extraArgs) throws Exception {
     String uri =
         String.format("ldap://localhost:%s", serverRule.getLdapServer().getPort());
     String dn = "cn=#UID,ou=Users,dc=myorg,dc=com";
-    String ldapArgs = String.format(
-        "--enable_ldap_auth --ldap_uri='%s' --ldap_bind_pattern='%s' " +
-        "--ldap_passwords_in_clear_ok --authorized_proxy_user_config=%s=%s",
-        uri, dn, testUser_, delegateUser_);
+    String ldapArgs =
+        String.format("--enable_ldap_auth --ldap_uri='%s' --ldap_bind_pattern='%s' "
+                + "--ldap_passwords_in_clear_ok %s",
+            uri, dn, extraArgs);
     int ret = CustomClusterRunner.StartImpalaCluster(ldapArgs);
     assertEquals(ret, 0);
   }
 
-  @After
-  public void cleanUp() throws Exception {
+  @AfterClass
+  public static void cleanUp() throws Exception {
     CustomClusterRunner.StartImpalaCluster();
   }
 
@@ -101,6 +106,7 @@ public class LdapImpalaShellTest {
    */
   @Test
   public void testShellLdapAuth() throws Exception {
+    setUp("");
     String query = "select logged_in_user()";
     // Templated shell commands to test a simple 'show tables' command.
     // 1. Valid username and password. Should succeed.
@@ -151,6 +157,8 @@ public class LdapImpalaShellTest {
    */
   @Test
   public void testHttpImpersonation() throws Exception {
+    setUp(
+        String.format("--authorized_proxy_user_config=%s=%s", testUser_, delegateUser_));
     // Ignore the test if python SSLContext support is not available.
     Assume.assumeTrue(pythonSupportsSSLContext());
     String invalidDelegateUser = "invalid-delegate-user";
@@ -180,4 +188,40 @@ public class LdapImpalaShellTest {
         query, "hs2-http", testUser_, testPassword_, "/?doAs=" + delegateUser_);
     RunShellCommand.Run(command, /* shouldSucceed */ true, delegateUser_, "");
   }
+
+  /**
+   * Tests the LDAP user and group filter configs.
+   */
+  @Test
+  public void testLdapFilters() throws Exception {
+    String groupDN = "cn=%s,ou=Groups,dc=myorg,dc=com";
+    // These correspond to the values in fe/src/test/resources/users.ldif
+    setUp(String.format("--ldap_group_filter=%s,another-group "
+            + "--ldap_user_filter=%s,%s,another-user "
+            + "--ldap_group_dn_pattern=%s "
+            + "--ldap_group_membership_key=uniqueMember "
+            + "--ldap_group_class_key=groupOfUniqueNames",
+        testUserGroup_, testUser_, testUser3_, groupDN));
+    String query = "select logged_in_user()";
+
+    // Run with user that passes the group filter but not the user filter, should fail.
+    String[] command =
+        buildCommand(query, "hs2-http", testUser2_, testPassword2_, "/cliservice");
+    RunShellCommand.Run(
+        command, /* shouldSucceed */ false, "", "Not connected to Impala");
+
+    // Run with user that passes the user filter but not the group filter, should fail.
+    command = buildCommand(query, "hs2-http", testUser3_, testPassword3_, "/cliservice");
+    RunShellCommand.Run(
+        command, /* shouldSucceed */ false, "", "Not connected to Impala");
+
+    // Run with user that doesn't pass either filter, should fail.
+    command = buildCommand(query, "hs2-http", testUser4_, testPassword4_, "/cliservice");
+    RunShellCommand.Run(
+        command, /* shouldSucceed */ false, "", "Not connected to Impala");
+
+    // Run with user that passes both filters, should succeed.
+    command = buildCommand(query, "hs2-http", testUser_, testPassword_, "/cliservice");
+    RunShellCommand.Run(command, /* shouldSucceed */ true, testUser_, "");
+  }
 }
diff --git a/fe/src/test/resources/users.ldif b/fe/src/test/resources/users.ldif
index 0415efb..4fd8c4f 100644
--- a/fe/src/test/resources/users.ldif
+++ b/fe/src/test/resources/users.ldif
@@ -32,4 +32,30 @@ objectClass: top
 cn: Test2Ldap
 sn: Ldap
 uid: ldaptest2
-userPassword: abcde
\ No newline at end of file
+userPassword: abcde
+
+dn: cn=Test3Ldap,ou=Users,dc=myorg,dc=com
+objectClass: inetOrgPerson
+objectClass: organizationalPerson
+objectClass: person
+objectClass: top
+cn: Test3Ldap
+sn: Ldap
+uid: ldaptest3
+userPassword: 67890
+
+dn: cn=Test4Ldap,ou=Users,dc=myorg,dc=com
+objectClass: inetOrgPerson
+objectClass: organizationalPerson
+objectClass: person
+objectClass: top
+cn: Test4Ldap
+sn: Ldap
+uid: ldaptest4
+userPassword: fghij
+
+dn: cn=group1,ou=Groups,dc=myorg,dc=com
+objectClass: top
+objectClass: groupOfUniqueNames
+uniqueMember: cn=Test1Ldap,ou=Users,dc=myorg,dc=com
+uniqueMember: cn=Test2Ldap,ou=Users,dc=myorg,dc=com
\ No newline at end of file