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

[impala] branch master updated: IMPALA-9988 (part 2): Integrate ldap filters and impala.doas.user

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

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


The following commit(s) were added to refs/heads/master by this push:
     new e63bf9d  IMPALA-9988 (part 2): Integrate ldap filters and impala.doas.user
e63bf9d is described below

commit e63bf9d6c1b2a67f68057f2b8bf077aa7be27256
Author: Thomas Tauber-Marshall <tm...@cloudera.com>
AuthorDate: Mon Jul 27 17:00:21 2020 -0700

    IMPALA-9988 (part 2): Integrate ldap filters and impala.doas.user
    
    This patch fixes the integration between LDAP filters and proxy
    users by ensuring that the 'impala.doas.user' HS2 config option is
    considered when applying filters. This requires deferring checking the
    filters until the OpenSession() call.
    
    This patch also introduces new flags --ldap_bind_dn and
    --ldap_bind_password_cmd which must be specified in order to use LDAP
    filters, unless the LDAP server is set up to allow anonymous binds.
    
    It also uses some gflag utilities from Kudu to tag
    --ldap_bind_password_cmd as sensitive and redact it on the webui and
    in logging in order to increase security in case a user specifies it
    as 'echo <password>'
    
    These config options are modeled after equivalent options in Hue:
    https://github.com/cloudera/hue/blob/master/desktop/conf.dist/hue.ini#L425
    
    Testing:
    - Added a test that uses the 'impala.doas.user' config with LDAP
      filters.
    
    Change-Id: I9ca8e1a0466288225efbe05b2d0068b8241df070
    Reviewed-on: http://gerrit.cloudera.org:8080/16252
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/common/logging.cc                           |  6 +-
 be/src/rpc/authentication.cc                       | 30 +++++++-
 be/src/service/impala-hs2-server.cc                | 21 ++++-
 be/src/service/impala-server.cc                    | 17 ++++
 be/src/service/impala-server.h                     |  3 +
 be/src/util/default-path-handlers.cc               |  5 +-
 be/src/util/ldap-util.cc                           | 90 +++++++++++++++++-----
 be/src/util/ldap-util.h                            | 18 +++--
 be/src/util/webserver.cc                           |  4 +-
 .../apache/impala/customcluster/LdapHS2Test.java   | 68 +++++++++++++++-
 .../impala/customcluster/LdapImpalaShellTest.java  | 22 ++----
 .../impala/customcluster/LdapWebserverTest.java    |  5 +-
 .../java/org/apache/impala/testutil/LdapUtil.java  |  3 +
 13 files changed, 235 insertions(+), 57 deletions(-)

diff --git a/be/src/common/logging.cc b/be/src/common/logging.cc
index 297bedb..21b9ae9 100644
--- a/be/src/common/logging.cc
+++ b/be/src/common/logging.cc
@@ -32,6 +32,7 @@
 #include <gutil/strings/substitute.h>
 
 #include "common/logging.h"
+#include "kudu/util/flags.h"
 #include "util/container-util.h"
 #include "util/debug-util.h"
 #include "util/error-util.h"
@@ -180,14 +181,15 @@ void impala::ShutdownLogging() {
 
 void impala::LogCommandLineFlags() {
   LOG(INFO) << "Flags (see also /varz are on debug webserver):" << endl
-            << google::CommandlineFlagsIntoString();
+            << kudu::CommandlineFlagsIntoString(kudu::EscapeMode::NONE);
 
   vector<google::CommandLineFlagInfo> flags;
   google::GetAllFlags(&flags, true);
   stringstream ss;
   for (const auto& flag: flags) {
     if (flag.hidden) {
-      ss << "--" << flag.name << "=" << flag.current_value << "\n";
+      string flag_value = CheckFlagAndRedact(flag, kudu::EscapeMode::NONE);
+      ss << "--" << flag.name << "=" << flag_value << "\n";
     }
   }
   string experimental_flags = ss.str();
diff --git a/be/src/rpc/authentication.cc b/be/src/rpc/authentication.cc
index 801d99e..e188f86 100644
--- a/be/src/rpc/authentication.cc
+++ b/be/src/rpc/authentication.cc
@@ -46,6 +46,7 @@
 #include "rpc/auth-provider.h"
 #include "rpc/cookie-util.h"
 #include "rpc/thrift-server.h"
+#include "runtime/exec-env.h"
 #include "transport/THttpServer.h"
 #include "transport/TSaslClientTransport.h"
 #include "util/auth-util.h"
@@ -177,6 +178,29 @@ static int SaslLogCallback(void* context, int level, const char* message) {
   return SASL_OK;
 }
 
+// Calls into the LDAP utils to check the provided user/pass.
+bool DoLdapCheck(const char* user, const char* pass, unsigned passlen) {
+  ImpalaLdap* ldap = AuthManager::GetInstance()->GetLdap();
+  bool success = ldap->LdapCheckPass(user, pass, passlen);
+
+  if (success) {
+    ImpalaServer* server = ExecEnv::GetInstance()->impala_server();
+    if (server == nullptr) {
+      LOG(FATAL) << "Invalid config: SASL LDAP is only supported for client connections "
+                 << "to an impalad.";
+    }
+    // If the user is an authorized proxy user, we do not yet know the effective user as
+    // it may be set by 'impala.doas.user', in which case we defer checking LDAP filters
+    // until OpenSession(). Otherwise, we prefer to check the filters as easly as
+    // possible, so check them here.
+    if (!server->IsAuthorizedProxyUser(user)) {
+      success = ldap->LdapCheckFilters(user);
+    }
+  }
+
+  return success;
+}
+
 // Wrapper around the function we use to check passwords with LDAP which has the function
 // signature required to work with SASL.
 //
@@ -189,8 +213,7 @@ static int SaslLogCallback(void* context, int level, const char* message) {
 // 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 AuthManager::GetInstance()->GetLdap()->LdapCheckPass(user, pass, passlen) ?
-      SASL_OK : SASL_FAIL;
+  return DoLdapCheck(user, pass, passlen) ? SASL_OK : SASL_FAIL;
 }
 
 // Sasl wants a way to ask us about some options, this function provides
@@ -445,8 +468,7 @@ bool BasicAuth(ThriftServer::ConnectionContext* connection_context,
   }
   string username = decoded.substr(0, colon);
   string password = decoded.substr(colon + 1);
-  bool ret = AuthManager::GetInstance()->GetLdap()->LdapCheckPass(username.c_str(),
-      password.c_str(), password.length(), connection_context->do_as_user);
+  bool ret = DoLdapCheck(username.c_str(), password.c_str(), password.length());
   if (ret) {
     // Authenication was successful, so set the username on the connection.
     connection_context->username = username;
diff --git a/be/src/service/impala-hs2-server.cc b/be/src/service/impala-hs2-server.cc
index 64e945b..6cccf28 100644
--- a/be/src/service/impala-hs2-server.cc
+++ b/be/src/service/impala-hs2-server.cc
@@ -90,6 +90,7 @@ const TProtocolVersion::type MAX_SUPPORTED_HS2_VERSION =
     } \
   } while (false)
 
+DECLARE_bool(enable_ldap_auth);
 DECLARE_string(hostname);
 DECLARE_int32(webserver_port);
 DECLARE_int32(idle_session_timeout);
@@ -296,8 +297,8 @@ void ImpalaServer::OpenSession(TOpenSessionResp& return_val,
 
   // DO NOT log this Thrift struct in its entirety, in case a bad client sets the
   // password.
-  VLOG_QUERY << "Opening session: " << PrintId(session_id) << " username: "
-             << request.username;
+  VLOG_QUERY << "Opening session: " << PrintId(session_id)
+             << " request username: " << request.username;
 
   // create a session state: initialize start time, session type, database and default
   // query options.
@@ -363,6 +364,18 @@ void ImpalaServer::OpenSession(TOpenSessionResp& return_val,
       }
     }
   }
+
+  // If the connected user is an authorized proxy user, we were not able to check the LDAP
+  // filters when the connection was created because we didn't know what the effective
+  // user would be yet, so check now.
+  if (FLAGS_enable_ldap_auth && IsAuthorizedProxyUser(state->connected_user)) {
+    bool success =
+        AuthManager::GetInstance()->GetLdap()->LdapCheckFilters(GetEffectiveUser(*state));
+    if (!success) {
+      HS2_RETURN_ERROR(return_val, "User is not authorized.", SQLSTATE_GENERAL_ERROR);
+    }
+  }
+
   RegisterSessionTimeout(state->session_timeout);
   TQueryOptionsToMap(state->QueryOptions(), &return_val.configuration);
 
@@ -389,8 +402,8 @@ void ImpalaServer::OpenSession(TOpenSessionResp& return_val,
   return_val.__isset.configuration = true;
   return_val.status.__set_statusCode(thrift::TStatusCode::SUCCESS_STATUS);
   return_val.serverProtocolVersion = state->hs2_version;
-  VLOG_QUERY << "Opened session: " << PrintId(session_id) << " username: "
-             << request.username;
+  VLOG_QUERY << "Opened session: " << PrintId(session_id)
+             << " effective username: " << GetEffectiveUser(*state);
 }
 
 void ImpalaServer::CloseSession(TCloseSessionResp& return_val,
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index d4f05c8..d939ef4 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -1729,6 +1729,23 @@ Status ImpalaServer::AuthorizeProxyUser(const string& user, const string& do_as_
   return Status::Expected(error_msg_str);
 }
 
+bool ImpalaServer::IsAuthorizedProxyUser(const string& user) {
+  if (user.empty()) return false;
+
+  // Get the short version of the user name (the user name up to the first '/' or '@')
+  // from the full principal name.
+  size_t end_idx = min(user.find("/"), user.find("@"));
+  // If neither are found (or are found at the beginning of the user name),
+  // return the username. Otherwise, return the username up to the matching character.
+  string short_user(
+      end_idx == string::npos || end_idx == 0 ? user : user.substr(0, end_idx));
+
+  return authorized_proxy_user_config_.find(short_user)
+      != authorized_proxy_user_config_.end()
+      || authorized_proxy_group_config_.find(short_user)
+      != authorized_proxy_group_config_.end();
+}
+
 void ImpalaServer::CatalogUpdateVersionInfo::UpdateCatalogVersionMetrics()
 {
   ImpaladMetrics::CATALOG_VERSION->SetValue(catalog_version);
diff --git a/be/src/service/impala-server.h b/be/src/service/impala-server.h
index f0236fb..dab8b29 100644
--- a/be/src/service/impala-server.h
+++ b/be/src/service/impala-server.h
@@ -501,6 +501,9 @@ class ImpalaServer : public ImpalaServiceIf,
   /// Appends the lineage_entry to lineage_logger_.
   Status AppendLineageEntry(const std::string& lineage_entry);
 
+  /// Returns true if 'user' was configured as an authorized proxy user.
+  bool IsAuthorizedProxyUser(const std::string& user) WARN_UNUSED_RESULT;
+
   // Mapping between query option names and levels
   QueryOptionLevels query_option_levels_;
 
diff --git a/be/src/util/default-path-handlers.cc b/be/src/util/default-path-handlers.cc
index f9c0cd5..c14491a 100644
--- a/be/src/util/default-path-handlers.cc
+++ b/be/src/util/default-path-handlers.cc
@@ -25,6 +25,7 @@
 #include <gutil/strings/substitute.h>
 
 #include "common/logging.h"
+#include "kudu/util/flags.h"
 #include "rpc/jni-thrift-util.h"
 #include "runtime/exec-env.h"
 #include "runtime/mem-tracker.h"
@@ -101,6 +102,8 @@ void FlagsHandler(const Webserver::WebRequest& req, Document* document) {
   GetAllFlags(&flag_info, true);
   Value flag_arr(kArrayType);
   for (const CommandLineFlagInfo& flag: flag_info) {
+    string flag_value = CheckFlagAndRedact(flag, kudu::EscapeMode::NONE);
+
     Value flag_val(kObjectType);
     Value name(flag.name.c_str(), document->GetAllocator());
     flag_val.AddMember("name", name, document->GetAllocator());
@@ -114,7 +117,7 @@ void FlagsHandler(const Webserver::WebRequest& req, Document* document) {
     Value default_value(flag.default_value.c_str(), document->GetAllocator());
     flag_val.AddMember("default", default_value, document->GetAllocator());
 
-    Value current_value(flag.current_value.c_str(), document->GetAllocator());
+    Value current_value(flag_value.c_str(), document->GetAllocator());
     flag_val.AddMember("current", current_value, document->GetAllocator());
 
     flag_val.AddMember("experimental", flag.hidden, document->GetAllocator());
diff --git a/be/src/util/ldap-util.cc b/be/src/util/ldap-util.cc
index b8adeb6..886e196 100644
--- a/be/src/util/ldap-util.cc
+++ b/be/src/util/ldap-util.cc
@@ -24,6 +24,8 @@
 #include <gutil/strings/util.h>
 
 #include "common/logging.h"
+#include "kudu/util/flag_tags.h"
+#include "util/os-util.h"
 
 #include "common/names.h"
 
@@ -55,7 +57,21 @@ DEFINE_string(ldap_group_membership_key, "member",
 DEFINE_string(ldap_group_class_key, "groupOfNames",
     "The LDAP objectClass each of the groups in --ldap_group_filter implements in LDAP.");
 
+DEFINE_string(ldap_bind_dn, "",
+    "Distinguished name of the user to bind as when doing user or group searches. Only "
+    "required if user or group filters are being used and the LDAP server is not "
+    "configured to allow anonymous searches.");
+DEFINE_string(ldap_bind_password_cmd, "",
+    "A Unix command whose output returns the password to use with --ldap_bind_dn. The "
+    "output of the command will be truncated to 1024 bytes and trimmed of trailing "
+    "whitespace.");
+TAG_FLAG(ldap_bind_password_cmd, sensitive);
+
 DECLARE_string(ldap_ca_certificate);
+DECLARE_string(ldap_user_filter);
+DECLARE_string(ldap_group_filter);
+DECLARE_string(principal);
+DECLARE_bool(skip_external_kerberos_auth);
 
 using boost::algorithm::replace_all;
 using namespace strings;
@@ -113,6 +129,13 @@ Status ImpalaLdap::ValidateFlags() {
                  << "hence passwords) could be intercepted by a "
                  << "man-in-the-middle attack";
   }
+
+  if ((!FLAGS_ldap_user_filter.empty() || !FLAGS_ldap_group_filter.empty())
+      && (!FLAGS_principal.empty() && !FLAGS_skip_external_kerberos_auth)) {
+    return Status("LDAP user and group filters may not be used if Kerberos auth is "
+                  "turned on for external connections.");
+  }
+
   return Status::OK();
 }
 
@@ -143,18 +166,37 @@ Status ImpalaLdap::Init(const std::string& user_filter, const std::string& group
     }
   }
 
+  if (!FLAGS_ldap_bind_password_cmd.empty()) {
+    if (!RunShellProcess(
+            FLAGS_ldap_bind_password_cmd, &bind_password_, true, {"JAVA_TOOL_OPTIONS"})) {
+      return Status(
+          Substitute("ldap_bind_password_cmd failed with output: '$0'", bind_password_));
+    }
+  }
+
   return Status::OK();
 }
 
-bool ImpalaLdap::LdapCheckPass(
-    const char* user, const char* pass, unsigned passlen, string do_as_user) {
+bool ImpalaLdap::LdapCheckPass(const char* user, const char* pass, unsigned passlen) {
   if (passlen == 0 && !FLAGS_ldap_allow_anonymous_binds) {
     // Disable anonymous binds.
     return false;
   }
 
+  string user_dn = ConstructUserDN(user);
   LDAP* ld;
-  int rc = ldap_initialize(&ld, FLAGS_ldap_uri.c_str());
+  VLOG_QUERY << "Trying simple LDAP bind for: " << user_dn;
+  bool success = Bind(user_dn, pass, passlen, &ld);
+  if (success) {
+    ldap_unbind_ext(ld, nullptr, nullptr);
+  }
+  VLOG(2) << "LDAP bind successful";
+  return success;
+}
+
+bool ImpalaLdap::Bind(
+    const std::string& user_dn, const char* pass, unsigned passlen, 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);
@@ -163,62 +205,72 @@ bool ImpalaLdap::LdapCheckPass(
 
   // 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);
+  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);
+    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);
+      ldap_unbind_ext(*ld, nullptr, nullptr);
       return false;
     }
     VLOG(2) << "Started TLS connection with LDAP server: " << FLAGS_ldap_uri;
   }
 
-  string user_dn = ConstructUserDN(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_dn;
-
   rc = ldap_sasl_bind_s(
-      ld, user_dn.c_str(), LDAP_SASL_SIMPLE, &cred, nullptr, nullptr, nullptr);
+      *ld, user_dn.c_str(), LDAP_SASL_SIMPLE, &cred, nullptr, nullptr, nullptr);
   // Free ld
   if (rc != LDAP_SUCCESS) {
     LOG(WARNING) << "LDAP authentication failure for " << user_dn << " : "
                  << ldap_err2string(rc);
-    ldap_unbind_ext(ld, nullptr, nullptr);
+    ldap_unbind_ext(*ld, nullptr, nullptr);
+    return false;
+  }
+
+  return true;
+}
+
+bool ImpalaLdap::LdapCheckFilters(std::string username) {
+  if (user_filter_.empty() && group_filter_.empty()) return true;
+
+  VLOG(2) << "Checking LDAP filters for " << username;
+  if (username.empty()) {
+    LOG(WARNING) << "Failed to check LDAP filters: username empty.";
     return false;
   }
 
-  VLOG_QUERY << "LDAP bind successful";
+  LDAP* ld;
+  bool success =
+      Bind(FLAGS_ldap_bind_dn, bind_password_.c_str(), bind_password_.size(), &ld);
+  if (!success) return false;
 
-  string filter_user = do_as_user == "" ? user : do_as_user;
-  if (!user_filter_.empty() && user_filter_.count(filter_user) != 1) {
-    LOG(WARNING) << "LDAP authentication failure for " << user_dn << ". Bind was "
+  if (!user_filter_.empty() && user_filter_.count(username) != 1) {
+    LOG(WARNING) << "LDAP authentication failure for " << username << ". Bind was "
                  << "successful but user is not in the authorized user list.";
     ldap_unbind_ext(ld, nullptr, nullptr);
     return false;
   }
 
-  string filter_user_dn = do_as_user == nullptr ? user_dn : ConstructUserDN(do_as_user);
   if (!group_filter_.empty()) {
+    string filter_user_dn = ConstructUserDN(username);
     if (!CheckGroupMembership(ld, filter_user_dn)) {
-      LOG(WARNING) << "LDAP authentication failure for " << user_dn << ". Bind was "
+      LOG(WARNING) << "LDAP authentication failure for " << username << ". 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);
-
+  VLOG(2) << "LDAP filter check for " << username << " was successful.";
   return true;
 }
 
diff --git a/be/src/util/ldap-util.h b/be/src/util/ldap-util.h
index f624b9d..6b31756 100644
--- a/be/src/util/ldap-util.h
+++ b/be/src/util/ldap-util.h
@@ -44,17 +44,16 @@ class ImpalaLdap {
   /// user or group filters as appropriate. 'passlen' is the length of the password.
   /// Returns true if authentication is successful.
   ///
-  /// 'do_as_user' is an optional delegate user that 'username' is trying act as a proxy
-  /// for. If provided, the group and user filters will be applied to 'do_as_user' and not
-  /// 'username'.
-  ///
   /// 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,
-      std::string do_as_user = "") WARN_UNUSED_RESULT;
+  bool LdapCheckPass(
+      const char* username, const char* password, unsigned passlen) WARN_UNUSED_RESULT;
+
+  /// Returns true if 'username' passes the LDAP user and group filters, if configured.
+  bool LdapCheckFilters(std::string username) WARN_UNUSED_RESULT;
 
  private:
   /// If non-empty, only users in this set can successfully authenticate.
@@ -66,6 +65,13 @@ class ImpalaLdap {
   /// The base DNs to use when preforming group searches.
   std::vector<std::string> group_filter_dns_;
 
+  /// The output of --ldap_bind_password_cmd, if specified.
+  std::string bind_password_;
+
+  /// Attempts a bind with 'user' and 'pass'. Returns true if successful and the handle is
+  /// returned in 'ldap', in which case the caller must call 'ldap_unbind_ext' on 'ldap'.
+  bool Bind(const std::string& user_dn, const char* pass, unsigned passlen, LDAP** ldap);
+
   /// 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);
diff --git a/be/src/util/webserver.cc b/be/src/util/webserver.cc
index 8986dba..11e1879 100644
--- a/be/src/util/webserver.cc
+++ b/be/src/util/webserver.cc
@@ -784,8 +784,8 @@ Status Webserver::HandleBasic(struct sq_connection* connection,
   }
   string username = decoded.substr(0, colon);
   string password = decoded.substr(colon + 1);
-  bool ret = ldap_->LdapCheckPass(username.c_str(), password.c_str(), password.length());
-  if (ret) {
+  if (ldap_->LdapCheckPass(username.c_str(), password.c_str(), password.length())
+      && ldap_->LdapCheckFilters(username)) {
     request_info->remote_user = strdup(username.c_str());
     return Status::OK();
   }
diff --git a/fe/src/test/java/org/apache/impala/customcluster/LdapHS2Test.java b/fe/src/test/java/org/apache/impala/customcluster/LdapHS2Test.java
index 1789709..c170b8e 100644
--- a/fe/src/test/java/org/apache/impala/customcluster/LdapHS2Test.java
+++ b/fe/src/test/java/org/apache/impala/customcluster/LdapHS2Test.java
@@ -17,6 +17,7 @@
 
 package org.apache.impala.customcluster;
 
+import static org.apache.impala.testutil.LdapUtil.*;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -54,13 +55,13 @@ public class LdapHS2Test {
 
   Metrics metrics = new Metrics();
 
-  @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", uri, dn);
+            + "--ldap_bind_pattern='%s' --ldap_passwords_in_clear_ok %s ",
+        uri, dn, extraArgs);
     int ret = CustomClusterRunner.StartImpalaCluster(ldapArgs);
     assertEquals(ret, 0);
   }
@@ -120,6 +121,7 @@ public class LdapHS2Test {
    */
   @Test
   public void testHiveserver2() throws Exception {
+    setUp("");
     verifyMetrics(0, 0);
     THttpClient transport = new THttpClient("http://localhost:28000");
     Map<String, String> headers = new HashMap<String, String>();
@@ -242,4 +244,64 @@ public class LdapHS2Test {
       }
     }
   }
+
+  /**
+   * Test for the interaction between the HS2 'impala.doas.user' property and LDAP user
+   * and group filters.
+   */
+  @Test
+  public void testHS2Impersonation() throws Exception {
+    // These correspond to the values in fe/src/test/resources/users.ldif
+    // Sets up a cluster where TEST_USER_4 can act as a proxy for any other user but
+    // doesn't pass any filters themselves, TEST_USER_1 and TEST_USER_2 can pass the group
+    // filter, and TEST_USER_1 and TEST_USER_3 pass the user filter.
+    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 "
+            + "--authorized_proxy_user_config=%s=* "
+            + "--ldap_bind_dn=%s --ldap_bind_password_cmd='echo -n %s' ",
+        TEST_USER_GROUP, TEST_USER_1, TEST_USER_3, GROUP_DN_PATTERN, TEST_USER_4,
+        TEST_USER_DN_1, TEST_PASSWORD_1));
+
+    THttpClient transport = new THttpClient("http://localhost:28000");
+    Map<String, String> headers = new HashMap<String, String>();
+    // Authenticate as the proxy user 'Test4Ldap'
+    headers.put("Authorization", "Basic VGVzdDRMZGFwOmZnaGlq");
+    transport.setCustomHeaders(headers);
+    transport.open();
+    TCLIService.Iface client = new TCLIService.Client(new TBinaryProtocol(transport));
+
+    // Open a session without specifying a 'doas', should fail as the proxy user won't
+    // pass the filters.
+    TOpenSessionReq openReq = new TOpenSessionReq();
+    TOpenSessionResp openResp = client.OpenSession(openReq);
+    assertEquals(openResp.getStatus().getStatusCode(), TStatusCode.ERROR_STATUS);
+
+    // Open a session with a 'doas' that will pass both filters, should succeed.
+    Map<String, String> config = new HashMap<String, String>();
+    config.put("impala.doas.user", TEST_USER_1);
+    openReq.setConfiguration(config);
+    openResp = client.OpenSession(openReq);
+    assertEquals(openResp.getStatus().getStatusCode(), TStatusCode.SUCCESS_STATUS);
+    // Running a query should succeed.
+    TOperationHandle operationHandle = execAndFetch(
+        client, openResp.getSessionHandle(), "select logged_in_user()", "Test1Ldap");
+
+    // Open a session with a 'doas' that doesn't pass the user filter, should fail.
+    config.put("impala.doas.user", TEST_USER_2);
+    openResp = client.OpenSession(openReq);
+    assertEquals(openResp.getStatus().getStatusCode(), TStatusCode.ERROR_STATUS);
+
+    // Open a session with a 'doas' that doesn't pass the group filter, should fail.
+    config.put("impala.doas.user", TEST_USER_3);
+    openResp = client.OpenSession(openReq);
+    assertEquals(openResp.getStatus().getStatusCode(), TStatusCode.ERROR_STATUS);
+
+    // Open a session with a 'doas' that doesn't pass either filter, should fail.
+    config.put("impala.doas.user", TEST_USER_4);
+    openResp = client.OpenSession(openReq);
+    assertEquals(openResp.getStatus().getStatusCode(), TStatusCode.ERROR_STATUS);
+  }
 }
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 a561a5a..bcefb19 100644
--- a/fe/src/test/java/org/apache/impala/customcluster/LdapImpalaShellTest.java
+++ b/fe/src/test/java/org/apache/impala/customcluster/LdapImpalaShellTest.java
@@ -70,13 +70,6 @@ public class LdapImpalaShellTest {
     assertEquals(ret, 0);
   }
 
-  @AfterClass
-  public static void cleanUp() throws Exception {
-    CustomClusterRunner.StartImpalaCluster();
-  }
-
-  /**
-
   /**
    * Checks if the local python supports SSLContext needed by shell http
    * transport tests. Python version shipped with CentOS6 is known to
@@ -182,14 +175,15 @@ public class LdapImpalaShellTest {
    */
   @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",
-        TEST_USER_GROUP, TEST_USER_1, TEST_USER_3, groupDN));
+            + "--ldap_group_class_key=groupOfUniqueNames "
+            + "--ldap_bind_dn=%s --ldap_bind_password_cmd='echo -n %s' ",
+        TEST_USER_GROUP, TEST_USER_1, TEST_USER_3, GROUP_DN_PATTERN, TEST_USER_DN_1,
+        TEST_PASSWORD_1));
     String query = "select logged_in_user()";
 
     // Run with user that passes the group filter but not the user filter, should fail.
@@ -222,7 +216,6 @@ public class LdapImpalaShellTest {
    */
   @Test
   public void testLdapFiltersWithProxy() throws Exception {
-    String groupDN = "cn=%s,ou=Groups,dc=myorg,dc=com";
     // These correspond to the values in fe/src/test/resources/users.ldif
     // Sets up a cluster where TEST_USER_4 can act as a proxy for any other user but
     // doesn't pass any filters themselves, TEST_USER_1 and TEST_USER_2 can pass the group
@@ -232,11 +225,12 @@ public class LdapImpalaShellTest {
             + "--ldap_group_dn_pattern=%s "
             + "--ldap_group_membership_key=uniqueMember "
             + "--ldap_group_class_key=groupOfUniqueNames "
-            + "--authorized_proxy_user_config=%s=*",
-        TEST_USER_GROUP, TEST_USER_1, TEST_USER_3, groupDN, TEST_USER_4));
+            + "--authorized_proxy_user_config=%s=* "
+            + "--ldap_bind_dn=%s --ldap_bind_password_cmd='echo -n %s' ",
+        TEST_USER_GROUP, TEST_USER_1, TEST_USER_3, GROUP_DN_PATTERN, TEST_USER_4,
+        TEST_USER_DN_1, TEST_PASSWORD_1));
 
     String query = "select logged_in_user()";
-
     // Run as the proxy user with a delegate that passes both filters, should succeed
     // and return the delegate user's name.
     String[] command = buildCommand(
diff --git a/fe/src/test/java/org/apache/impala/customcluster/LdapWebserverTest.java b/fe/src/test/java/org/apache/impala/customcluster/LdapWebserverTest.java
index aa49b54..3d7ac73 100644
--- a/fe/src/test/java/org/apache/impala/customcluster/LdapWebserverTest.java
+++ b/fe/src/test/java/org/apache/impala/customcluster/LdapWebserverTest.java
@@ -130,8 +130,9 @@ public class LdapWebserverTest {
             + "--webserver_ldap_user_filter=%s,%s "
             + "--ldap_group_dn_pattern=ou=Groups,dc=myorg,dc=com "
             + "--ldap_group_membership_key=uniqueMember "
-            + "--ldap_group_class_key=groupOfUniqueNames",
-        TEST_USER_GROUP, TEST_USER_1, TEST_USER_3), "");
+            + "--ldap_group_class_key=groupOfUniqueNames "
+            + "--ldap_bind_dn=%s --ldap_bind_password_cmd='echo -n %s' ",
+        TEST_USER_GROUP, TEST_USER_1, TEST_USER_3, TEST_USER_DN_1, TEST_PASSWORD_1), "");
     // start-impala-cluster contacts the webui to confirm the impalads have started, so
     // there will already be some successful auth attempts.
     verifyMetrics(Range.atLeast(1L), zero, Range.atLeast(1L), zero);
diff --git a/fe/src/test/java/org/apache/impala/testutil/LdapUtil.java b/fe/src/test/java/org/apache/impala/testutil/LdapUtil.java
index 8f448d2..b9026ea 100644
--- a/fe/src/test/java/org/apache/impala/testutil/LdapUtil.java
+++ b/fe/src/test/java/org/apache/impala/testutil/LdapUtil.java
@@ -20,6 +20,7 @@ package org.apache.impala.testutil;
 public class LdapUtil {
   // These correspond to the values in fe/src/test/resources/users.ldif
   public static final String TEST_USER_1 = "Test1Ldap";
+  public static final String TEST_USER_DN_1 = "cn=Test1Ldap,ou=Users,dc=myorg,dc=com";
   public static final String TEST_PASSWORD_1 = "12345";
 
   public static final String TEST_USER_2 = "Test2Ldap";
@@ -33,4 +34,6 @@ public class LdapUtil {
 
   // TEST_USER_1 and TEST_USER_2 are members of this group.
   public static final String TEST_USER_GROUP = "group1";
+
+  public static final String GROUP_DN_PATTERN = "cn=%s,ou=Groups,dc=myorg,dc=com";
 }