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 2023/03/12 23:54:59 UTC

[impala] branch master updated (b5524e95a -> 39fea06f2)

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

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


    from b5524e95a IMPALA-11935: Generate core dumps if ASAN/TSAN/UBSAN built be tests crash
     new 490dd7b11 IMPALA-11726: Allow LDAP user and group filter when Kerberos is enabled
     new 39fea06f2 IMPALA-11990: Make actual failures clearer

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 be/src/rpc/authentication-test.cc                  |  67 +++
 be/src/rpc/authentication.cc                       | 133 ++++-
 be/src/util/ldap-util.cc                           |   6 +-
 bin/rat_exclude_files.txt                          |   2 +
 docs/topics/impala_fixed_issues.xml                |   2 +-
 .../customcluster/KerberosKdcEnvironment.java      | 178 ++++++
 .../impala/customcluster/LdapImpalaShellTest.java  |  10 +-
 .../customcluster/LdapKerberosImpalaShellTest.java | 640 +++++++++++++++++++++
 .../LdapKerberosImpalaShellTestBase.java           | 123 ++++
 ...hBindDefaultFiltersKerberosImpalaShellTest.java | 165 ++++++
 .../LdapSearchBindImpalaShellTest.java             |  57 +-
 .../LdapSimpleBindImpalaShellTest.java             |  55 +-
 .../impala/customcluster/RunShellCommand.java      |  14 +-
 .../java/org/apache/impala/testutil/LdapUtil.java  |   2 +
 fe/src/test/resources/adschema.ldif                |  57 ++
 fe/src/test/resources/adusers.ldif                 |  59 ++
 fe/src/test/resources/users.ldif                   |  11 +
 testdata/bin/create-load-data.sh                   |   6 +-
 testdata/bin/kill-hbase.sh                         |   3 +-
 19 files changed, 1558 insertions(+), 32 deletions(-)
 create mode 100644 fe/src/test/java/org/apache/impala/customcluster/KerberosKdcEnvironment.java
 create mode 100644 fe/src/test/java/org/apache/impala/customcluster/LdapKerberosImpalaShellTest.java
 create mode 100644 fe/src/test/java/org/apache/impala/customcluster/LdapKerberosImpalaShellTestBase.java
 create mode 100644 fe/src/test/java/org/apache/impala/customcluster/LdapSearchBindDefaultFiltersKerberosImpalaShellTest.java
 create mode 100644 fe/src/test/resources/adschema.ldif
 create mode 100644 fe/src/test/resources/adusers.ldif


[impala] 01/02: IMPALA-11726: Allow LDAP user and group filter when Kerberos is enabled

Posted by st...@apache.org.
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

commit 490dd7b1151881f91d339b61104c35e59fb61814
Author: Gergely Farkas <gf...@cloudera.com>
AuthorDate: Tue Feb 28 12:27:15 2023 +0100

    IMPALA-11726: Allow LDAP user and group filter when Kerberos is enabled
    
    This change does two things for the Kerberos authentication support
    for impala clients:
    
    1) Introduces allow_custom_ldap_filters_with_kerberos_auth flag,
    which removes the restriction that prevents to use LDAP group/user
    search filters when Kerberos authentication is enabled. When the flag
    is set both Kerberos and LDAP can work with impala clients
    (impala-shell, jdbc, odbc, impyla) even if the group/user filters are
    defined. The flag default value is false, which ensures backwards
    compatibility.
    
    2) Introduces enable_group_filter_check_for_authenticated_kerberos_user
    flag, which allows group filters to be applied for non-proxy users
    that belong to the authenticated Kerberos principals.
    The verified username comes from the Kerberos principal: The username
    is the first member of the authenticated Kerberos principal, where the
    principal can be username/host@realm or username@realm.
    Regardless of whether the flag is enabled or not, LDAP filters are not
    applied for authorized proxy users (neither when using LDAP nor when
    using Kerberos authentication). In case of delegation, filters are
    applied for delegated users.
    This flag makes sense if Kerberos and LDAP authentication is enabled
    and the users in the KDC and LDAP are synchronized (e.g. Active
    Directory provides both LDAP and Kerberos authentication).
    The flag default value is false, which ensures backwards compatibility.
    
    Notes:
    
    If the allow_custom_ldap_filters_with_kerberos_auth flag is disabled,
    it is still possible to use LDAP and Kerberos authentication together,
    but in a limited way: Only LDAP search bind authentication mode can be
    used, where there are default user and group search filters (that are
    defined for Active Directory LDAP schema). One major limitation here
    - apart from the AD directory schema assumed in the default filters -
    is that the only possibility to control user access is to select the
    appropriate user and group search base dn (e.g. granting LDAP access
    to users/groups defined in a given subtree)
    Even in this edge case, it is still allowed to enable the
    enable_group_filter_check_for_authenticated_kerberos_user flag. If this
    happens, then the default filters in LDAP search bind will be applied
    for Kerberos authenticated non-proxy users.
    
    Another edge case where the LDAP authentication is enabled, the
    user access is controlled by custom LDAP filters (LDAP auth only),
    and the external Kerberos authentication is also enabled, but the users
    in KDC and LDAP are not in sync:
    In this case the allow_custom_ldap_filters_with_kerberos_auth flag must
    be set, but enable_group_filter_check_for_authenticated_kerberos_user
    flag should be disabled, otherwise an unauthorized response may be
    received during Kerberos authentication (depending on whether the
    authenticated Kerberos user passes the custom LDAP filters or not).
    In such cases, access to Kerberos users must be controlled by other
    ways (e.g. within FreeIPA KDC with host-based access control rules).
    
    Tests:
    - New unit test created to check the behavior of AuthManager with
      and without allow_custom_ldap_filters_with_kerberos_auth flag.
    - New custom cluster tests created:
      - impala-shell tests that validate existing LDAP search bind
        and simple bind functionality with Kerberos authentication
        enabled (LdapSearchBindImpalaShellTest and
        LdapSimpleBindImpalaShellTest suites are now parameterized),
      - impala-shell tests that validate backwards compatibility
        when allow_custom_ldap_filters_with_kerberos_auth flag and
        enable_group_filter_check_for_authenticated_kerberos_user
        flags are disabled
        (LdapSearchBindDefaultFiltersKerberosImpalaShellTest)
      - various impala-shell tests that validate Kerberos
        authentication in an environment where LDAP authentication
        is also enabled (LdapKerberosImpalaShellTest)
    - Manual tests with a snapshot build in CDP PVC DS with LDAP and
      Kerberos authentication enabled, user and group filters provided.
    
    Change-Id: If3ca9c4ff8a17167e5233afabdd14c948edb46de
    Reviewed-on: http://gerrit.cloudera.org:8080/19561
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/rpc/authentication-test.cc                  |  67 +++
 be/src/rpc/authentication.cc                       | 133 ++++-
 be/src/util/ldap-util.cc                           |   6 +-
 bin/rat_exclude_files.txt                          |   2 +
 .../customcluster/KerberosKdcEnvironment.java      | 178 ++++++
 .../impala/customcluster/LdapImpalaShellTest.java  |  10 +-
 .../customcluster/LdapKerberosImpalaShellTest.java | 640 +++++++++++++++++++++
 .../LdapKerberosImpalaShellTestBase.java           | 123 ++++
 ...hBindDefaultFiltersKerberosImpalaShellTest.java | 165 ++++++
 .../LdapSearchBindImpalaShellTest.java             |  57 +-
 .../LdapSimpleBindImpalaShellTest.java             |  55 +-
 .../impala/customcluster/RunShellCommand.java      |  14 +-
 .../java/org/apache/impala/testutil/LdapUtil.java  |   2 +
 fe/src/test/resources/adschema.ldif                |  57 ++
 fe/src/test/resources/adusers.ldif                 |  59 ++
 fe/src/test/resources/users.ldif                   |  11 +
 16 files changed, 1552 insertions(+), 27 deletions(-)

diff --git a/be/src/rpc/authentication-test.cc b/be/src/rpc/authentication-test.cc
index 930b4e143..b2c78cfcd 100644
--- a/be/src/rpc/authentication-test.cc
+++ b/be/src/rpc/authentication-test.cc
@@ -18,9 +18,11 @@
 #include "testutil/gtest-util.h"
 #include "common/init.h"
 #include "common/logging.h"
+#include "kudu/security/test/mini_kdc.h"
 #include "rpc/authentication.h"
 #include "rpc/thrift-server.h"
 #include "util/auth-util.h"
+#include "util/kudu-status-util.h"
 #include "util/network-util.h"
 #include "util/openssl-util.h"
 #include "util/thread.h"
@@ -35,6 +37,11 @@ DECLARE_string(ssl_client_ca_certificate);
 DECLARE_string(ssl_server_certificate);
 DECLARE_string(ssl_private_key);
 DECLARE_string(internal_principals_whitelist);
+DECLARE_bool(allow_custom_ldap_filters_with_kerberos_auth);
+DECLARE_bool(ldap_search_bind_authentication);
+DECLARE_string(ldap_user_search_basedn);
+DECLARE_string(ldap_user_filter);
+DECLARE_string(ldap_group_filter);
 
 // These are here so that we can grab them early in main() - the kerberos
 // init can clobber KRB5_KTNAME in PrincipalSubstitution.
@@ -213,6 +220,66 @@ TEST(Auth, BadPrincipalFormat) {
   EXPECT_ERROR(sa.InitKerberos("service_name/localhost"), 2);
 }
 
+// Set up ldap and kerberos flags and
+// 1. check if there is an error when specifying custom search filters,
+// 2. and the error does not occur when the
+//    allow_custom_ldap_filters_with_kerberos_auth flag is turned on.
+TEST(Auth, LdapKerbAuthCustomFiltersNotAllowed) {
+  AuthProvider* ap = NULL;
+  SecureAuthProvider* sa = NULL;
+
+  // Initialize the mini kdc.
+  kudu::MiniKdc kdc(kudu::MiniKdcOptions{});
+  KUDU_ASSERT_OK(kdc.Start());
+  kdc.SetKrb5Environment();
+  string kt_path;
+  KUDU_ASSERT_OK(kdc.CreateServiceKeytab("HTTP/127.0.0.1", &kt_path));
+  CHECK_ERR(setenv("KRB5_KTNAME", kt_path.c_str(), 1));
+  KUDU_ASSERT_OK(kdc.CreateUserPrincipal("alice"));
+  KUDU_ASSERT_OK(kdc.Kinit("alice"));
+
+  // Set up a fake impala server with Kerberos enabled.
+  gflags::FlagSaver saver;
+
+  FLAGS_principal = "HTTP/127.0.0.1@KRBTEST.COM";
+  FLAGS_keytab_file = kt_path;
+  FLAGS_enable_ldap_auth = true;
+  FLAGS_ldap_uri = "ldaps://bogus.com";
+  FLAGS_skip_external_kerberos_auth = false; // external Kerberos auth enabled
+  FLAGS_skip_internal_kerberos_auth = true; // internal Kerberos auth disabled (no auth)
+  FLAGS_ldap_user_search_basedn = "dc=kbrtest,dc=com";
+  FLAGS_ldap_search_bind_authentication = true;
+  FLAGS_ldap_user_filter = "(&(objectClass=user)(sAMAccountName={0}))";
+
+  // Initialization based on above "command line" args should fail
+  // due to the custom ldap search filters specified.
+  ASSERT_ERROR_MSG(AuthManager::GetInstance()->Init(),
+      "LDAP user and group filters may not be used "
+      "if Kerberos auth is turned on for external connections.");
+
+  // Initialization based on above "command line" args should not fail
+  // if the use of custom ldap filters is enabled with Kerberos auth.
+  FLAGS_allow_custom_ldap_filters_with_kerberos_auth = true;
+  ASSERT_OK(AuthManager::GetInstance()->Init());
+
+  // External auth provider is sasl, ldap, and kerberos
+  ap = AuthManager::GetInstance()->GetExternalAuthProvider();
+  ASSERT_TRUE(ap->is_secure());
+  sa = dynamic_cast<SecureAuthProvider*>(ap);
+  ASSERT_TRUE(sa->has_ldap());
+  ASSERT_EQ(FLAGS_principal, sa->principal());
+
+  ap = AuthManager::GetInstance()->GetExternalHttpAuthProvider();
+  ASSERT_TRUE(ap->is_secure());
+  sa = dynamic_cast<SecureAuthProvider*>(ap);
+  ASSERT_TRUE(sa->has_ldap());
+  ASSERT_EQ(FLAGS_principal, sa->principal());
+
+  // Internal auth provider is not secure (NoAuthProvider)
+  ap = AuthManager::GetInstance()->GetInternalAuthProvider();
+  ASSERT_FALSE(ap->is_secure());
+}
+
 }
 int main(int argc, char** argv) {
   ::testing::InitGoogleTest(&argc, argv);
diff --git a/be/src/rpc/authentication.cc b/be/src/rpc/authentication.cc
index b11565243..4f3b4ad94 100644
--- a/be/src/rpc/authentication.cc
+++ b/be/src/rpc/authentication.cc
@@ -194,6 +194,13 @@ DEFINE_bool_hidden(jwt_allow_without_tls, false,
     "When this configuration is set to true, Impala allows JWT authentication on "
     "unsecure channel. This should be only enabled for testing, or development for which "
     "TLS is handled by proxy.");
+DEFINE_bool(enable_group_filter_check_for_authenticated_kerberos_user, false,
+    "If this configuration is set to true, Impala checks the provided "
+    "LDAP group filter, if any, with the authenticated Kerberos user. "
+    "This should be only enabled if both Kerberos and LDAP authentication are enabled "
+    "and the users in KDC and LDAP are synchronized (e.g. when the KDC and the LDAP "
+    "is the same Active Directory server). "
+    "The default value is false, which provides backwards-compatible behavior.");
 
 namespace impala {
 
@@ -268,27 +275,34 @@ static int SaslLogCallback(void* context, int level, const char* message) {
   return SASL_OK;
 }
 
-// Calls into the LDAP utils to check the provided user/pass.
+// Calls into the LDAP utils to check the provided group filters
+bool DoLdapCheckFilters(const char* user) {
+  ImpalaLdap* ldap = AuthManager::GetInstance()->GetLdap();
+  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)) {
+    return true;
+  }
+  return ldap->LdapCheckFilters(user);
+}
+
+// Calls into the LDAP utils to check the provided user/pass,
+// and the provided group filters.
 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);
-    }
+  if (!success) {
+    return false;
   }
 
-  return success;
+  return DoLdapCheckFilters(user);
 }
 
 // Wrapper around the function we use to check passwords with LDAP which has the function
@@ -480,8 +494,66 @@ int SaslAuthorizeInternal(sasl_conn_t* conn, void* context,
   return SASL_BADAUTH;
 }
 
+// Takes a Kerberos principal (either user/hostname@realm or user@realm)
+// and returns the username part.
+string GetShortUsernameFromKerberosPrincipal(const string& principal) {
+  size_t end_idx = min(principal.find("/"), principal.find("@"));
+  string short_user(
+      end_idx == string::npos || end_idx == 0 ?
+      principal : principal.substr(0, end_idx));
+  return short_user;
+}
+
+// If Kerberos and LDAP authentications are enabled and
+// enable_group_filter_check_for_authenticated_kerberos_user flag is set,
+// then this callback checks if the authenticated user passes LDAP group
+// filters.
+//
+// conn: Sasl connection - Ignored
+// context: Ignored, always NULL
+// requested_user: The identity/username to authorize
+// rlen: Length of above
+// auth_identity: "The identity associated with the secret"
+// alen: Length of above
+// def_realm: Default user realm
+// urlen: Length of above
+// propctx: Auxiliary properties - Ignored
+// Return: SASL_OK
+static int SaslKerberosAuthorizeExternal(sasl_conn_t* conn, void* context,
+    const char* requested_user, unsigned rlen,
+    const char* auth_identity, unsigned alen,
+    const char* def_realm, unsigned urlen,
+    struct propctx* propctx) {
+  if (FLAGS_enable_ldap_auth &&
+      FLAGS_enable_group_filter_check_for_authenticated_kerberos_user) {
+    DCHECK(IsKerberosEnabled());
+
+    string username = string(requested_user, rlen);
+    string short_user =
+        GetShortUsernameFromKerberosPrincipal(username);
+
+    LOG(INFO) << "Checking LDAP group filters for "
+              << "username \"" << short_user << "\" "
+              << "parsed from user principal \""
+              << username << "\".";
+
+    bool success = DoLdapCheckFilters(short_user.c_str());
+    if (!success) {
+      LOG(WARNING) << "Got authenticated principal but the "
+                   << "user \"" << short_user << "\" "
+                   << "didn't pass the group filters.";
+      return SASL_BADAUTH;
+    }
+  }
+
+  LOG(INFO) << "Successfully authenticated client user \""
+            << string(requested_user, rlen) << "\"";
+  return SASL_OK;
+}
+
 // This callback could be used to authorize or restrict access to certain
-// users.  Currently it is used to log a message that we successfully
+// users when authenticating with LDAP.
+// Currently it is used to log a message that we successfully
 // authenticated with a user on an external connection.
 //
 // conn: Sasl connection - Ignored
@@ -494,7 +566,7 @@ int SaslAuthorizeInternal(sasl_conn_t* conn, void* context,
 // urlen: Length of above
 // propctx: Auxiliary properties - Ignored
 // Return: SASL_OK
-static int SaslAuthorizeExternal(sasl_conn_t* conn, void* context,
+static int SaslLdapAuthorizeExternal(sasl_conn_t* conn, void* context,
     const char* requested_user, unsigned rlen,
     const char* auth_identity, unsigned alen,
     const char* def_realm, unsigned urlen,
@@ -670,6 +742,27 @@ bool NegotiateAuth(ThriftServer::ConnectionContext* connection_context,
                     << TNetworkAddressToString(connection_context->network_address)
                     << ": " << spnego_status.ToString();
       } else {
+        if (FLAGS_enable_ldap_auth &&
+            FLAGS_enable_group_filter_check_for_authenticated_kerberos_user) {
+
+          string short_user = GetShortUsernameFromKerberosPrincipal(username);
+
+          LOG(INFO) << "Checking LDAP group filters for "
+                    << "username \"" << short_user << "\" "
+                    << "parsed from user principal \""
+                    << username << "\".";
+
+          bool success = DoLdapCheckFilters(short_user.c_str());
+          if (!success) {
+            LOG(WARNING) << "Got authenticated principal for SPNEGO-authenticated "
+                        << "connection from "
+                        << TNetworkAddressToString(connection_context->network_address)
+                        << " but the authenticated user \"" << short_user << "\" "
+                        << "didn't pass the group filters.";
+            return false;
+          }
+        }
+
         // Authentication was successful, so set the username on the connection.
         connection_context->username = username;
         // Create a cookie to return.
@@ -915,7 +1008,7 @@ Status InitAuth(const string& appname) {
       KERB_EXT_CALLBACKS[0].context = ((void *)"Kerberos (external)");
 
       KERB_EXT_CALLBACKS[1].id = SASL_CB_PROXY_POLICY;
-      KERB_EXT_CALLBACKS[1].proc = (int (*)())&SaslAuthorizeExternal;
+      KERB_EXT_CALLBACKS[1].proc = (int (*)())&SaslKerberosAuthorizeExternal;
       KERB_EXT_CALLBACKS[1].context = NULL;
 
       KERB_EXT_CALLBACKS[2].id = SASL_CB_LIST_END;
@@ -930,7 +1023,7 @@ Status InitAuth(const string& appname) {
       LDAP_EXT_CALLBACKS[0].context = ((void *)"LDAP");
 
       LDAP_EXT_CALLBACKS[1].id = SASL_CB_PROXY_POLICY;
-      LDAP_EXT_CALLBACKS[1].proc = (int (*)())&SaslAuthorizeExternal;
+      LDAP_EXT_CALLBACKS[1].proc = (int (*)())&SaslLdapAuthorizeExternal;
       LDAP_EXT_CALLBACKS[1].context = NULL;
 
       // This last callback is where we take the password and turn around and
diff --git a/be/src/util/ldap-util.cc b/be/src/util/ldap-util.cc
index f95796313..d90c8aa83 100644
--- a/be/src/util/ldap-util.cc
+++ b/be/src/util/ldap-util.cc
@@ -50,6 +50,9 @@ 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.");
+DEFINE_bool(allow_custom_ldap_filters_with_kerberos_auth, false,
+    "If set, will allow custom LDAP user and group filters even if Kerberos "
+    "authentication is enabled. Disabled by default.");
 TAG_FLAG(ldap_bind_password_cmd, sensitive);
 
 DECLARE_string(ldap_ca_certificate);
@@ -111,7 +114,8 @@ Status ImpalaLdap::ValidateFlags() {
   }
 
   if ((!FLAGS_ldap_user_filter.empty() || !FLAGS_ldap_group_filter.empty())
-      && (!FLAGS_principal.empty() && !FLAGS_skip_external_kerberos_auth)) {
+      && (!FLAGS_principal.empty() && !FLAGS_skip_external_kerberos_auth
+          && !FLAGS_allow_custom_ldap_filters_with_kerberos_auth)) {
     return Status("LDAP user and group filters may not be used if Kerberos auth is "
                   "turned on for external connections.");
   }
diff --git a/bin/rat_exclude_files.txt b/bin/rat_exclude_files.txt
index 825eef9b5..7f828621c 100644
--- a/bin/rat_exclude_files.txt
+++ b/bin/rat_exclude_files.txt
@@ -128,6 +128,8 @@ be/src/kudu/util/testdata/*.txt
 be/src/testutil/*.pem
 *.json
 fe/src/test/resources/*.xml
+fe/src/test/resources/adschema.ldif
+fe/src/test/resources/adusers.ldif
 fe/src/test/resources/hbase-jaas-client.conf.template
 fe/src/test/resources/hbase-jaas-server.conf.template
 fe/src/test/resources/users.ldif
diff --git a/fe/src/test/java/org/apache/impala/customcluster/KerberosKdcEnvironment.java b/fe/src/test/java/org/apache/impala/customcluster/KerberosKdcEnvironment.java
new file mode 100644
index 000000000..5f393c3c2
--- /dev/null
+++ b/fe/src/test/java/org/apache/impala/customcluster/KerberosKdcEnvironment.java
@@ -0,0 +1,178 @@
+// 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.impala.customcluster;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.kerby.kerberos.kerb.KrbException;
+import org.apache.kerby.kerberos.kerb.client.KrbClient;
+import org.apache.kerby.kerberos.kerb.server.SimpleKdcServer;
+import org.apache.kerby.kerberos.kerb.type.ticket.TgtTicket;
+import org.apache.log4j.Logger;
+import org.junit.rules.ExternalResource;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Package protected helper class to encapsulate simple
+ * Kerberos KDC server used by junit tests.
+ */
+class KerberosKdcEnvironment extends ExternalResource {
+
+  private final Logger LOG =
+          Logger.getLogger(KerberosKdcEnvironment.class);
+
+  private final static String realm = "myorg.com";
+  private final static String servicePrincipal =
+          String.format("impala/localhost@%s", realm);
+
+  private final TemporaryFolder testFolder;
+  private SimpleKdcServer kerbyServer;
+
+  public KerberosKdcEnvironment(TemporaryFolder testFolder) {
+    this.testFolder = testFolder;
+  }
+
+  @Override
+  protected void before() throws Throwable {
+    testFolder.create();
+
+    kerbyServer = new SimpleKdcServer();
+    kerbyServer.setKdcRealm(realm);
+    kerbyServer.setAllowUdp(false);
+    kerbyServer.setWorkDir(testFolder.getRoot());
+    kerbyServer.init();
+
+    // Create service principal and keytab file for impala components
+    kerbyServer.createPrincipal(servicePrincipal, "password");
+    File keytabFile = new File(getServiceKeytabFilePath());
+    kerbyServer.exportPrincipal(servicePrincipal, keytabFile);
+
+    kerbyServer.start();
+  }
+
+  @Override
+  protected void after() {
+    try {
+      kerbyServer.stop();
+    } catch (KrbException e) {
+      LOG.error("An exception received while stopping KDC server", e);
+    }
+  }
+
+  public String getServicePrincipal() {
+    return servicePrincipal;
+  }
+
+  public String getServiceKeytabFilePath() throws IOException {
+    return new File(kerbyServer.getWorkDir().getCanonicalPath() + "/impala.keytab")
+            .getCanonicalPath();
+  }
+
+  public String getKrb5ConfigPath() throws IOException {
+    return kerbyServer.getWorkDir().getCanonicalPath() + "/krb5.conf";
+  }
+
+  public String getUserPrincipal(String user) {
+    return String.format("%s@%s", user, realm);
+  }
+
+  public String createUserPrincipalAndCredentialsCache(String username)
+          throws KrbException, IOException {
+    String userPrincipal = getUserPrincipal(username);
+    deleteUserPrincipalIfExists(userPrincipal);
+    kerbyServer.createPrincipal(userPrincipal, "password");
+
+    File credentialsCache = testFolder.newFile();
+
+    KrbClient krbClient = new KrbClient(kerbyServer.getWorkDir());
+    krbClient.init();
+    TgtTicket tgt = krbClient.requestTgt(username, "password" );
+    krbClient.storeTicket(tgt, credentialsCache);
+
+    return credentialsCache.getCanonicalPath();
+  }
+
+  private void deleteUserPrincipalIfExists(String userPrincipal )
+          throws KrbException {
+    if (kerbyServer.getIdentityService().getIdentity(userPrincipal) != null) {
+      kerbyServer.deletePrincipal(userPrincipal);
+    }
+  }
+
+  public Map<String, String> getKerberosAuthFlags()
+          throws IOException {
+    return ImmutableMap.of(
+            "principal", getServicePrincipal(), // enables Kerberos auth
+            "keytab_file", getServiceKeytabFilePath()
+    );
+  }
+
+  private Map<String, String> getClusterEnv() throws IOException {
+    String krb5Config = getKrb5ConfigPath();
+    return ImmutableMap.of(
+            "KRB5_CONFIG", krb5Config,
+            "KRB5_KDC_PROFILE", krb5Config
+    );
+  }
+
+  private String overrideKrbCcNameFlag(String args) throws IOException {
+    // in order to use a unique credentials cache file we need to
+    // override krb5_ccname flag
+    return String.format("%s --krb5_ccname=%s", args,
+            testFolder.newFile().getCanonicalPath());
+  }
+
+  public int startImpalaClusterWithArgs(String args)
+          throws IOException, InterruptedException {
+    // Note: To avoid race conditions between impala components each daemon uses
+    // its own unique krb credentials cache file when creating krb cc at initial kinit.
+    // For the same reason, the cluster size is limited to 1 instance of each daemon type.
+    return CustomClusterRunner.StartImpalaCluster(
+            overrideKrbCcNameFlag(args), // impalad args
+            overrideKrbCcNameFlag(args), // catalogd args
+            overrideKrbCcNameFlag(args), // statestored args
+            getClusterEnv(),
+            "--cluster_size=1");
+  }
+
+  public String[] getImpalaShellEnv(String credentialsCacheFilePath) throws IOException {
+    List<String> envList =
+            System.getenv().entrySet().stream()
+                    .map(entry -> String.format("%s=%s", entry.getKey(),
+                            entry.getValue()))
+                    .collect(Collectors.toList());
+
+    // Kerberos environment variables defined for impala-shell process:
+    // KRB5_CONFIG and KRB5_KDC_PROFILE env variables refer to the Kerberos configuration,
+    // KRB5CCNAME env variable refers to the credentials cache file of the provided user.
+    String krb5ConfigPath = getKrb5ConfigPath();
+    envList.addAll(
+            ImmutableList.of(
+                    String.format("KRB5_CONFIG=%s", krb5ConfigPath),
+                    String.format("KRB5_KDC_PROFILE=%s", krb5ConfigPath),
+                    String.format("KRB5CCNAME=FILE:%s", credentialsCacheFilePath)));
+    return envList.toArray(new String[0]);
+  }
+
+}
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 bd09aaef8..c6177a045 100644
--- a/fe/src/test/java/org/apache/impala/customcluster/LdapImpalaShellTest.java
+++ b/fe/src/test/java/org/apache/impala/customcluster/LdapImpalaShellTest.java
@@ -22,6 +22,8 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import com.google.common.collect.Range;
+
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
 import org.apache.directory.server.annotations.CreateLdapServer;
@@ -55,17 +57,21 @@ public class LdapImpalaShellTest {
     String ldapArgs = String.format("--enable_ldap_auth --ldap_uri='%s' "
             + "--ldap_passwords_in_clear_ok %s",
         uri, extraArgs);
-    int ret = CustomClusterRunner.StartImpalaCluster(ldapArgs);
+    int ret = startImpalaCluster(ldapArgs);
     assertEquals(ret, 0);
     verifyMetrics(zero, zero, zero, zero);
   }
 
+  protected int startImpalaCluster(String args) throws IOException, InterruptedException {
+    return CustomClusterRunner.StartImpalaCluster(args);
+  }
+
   /**
    * Checks if the local python supports SSLContext needed by shell http
    * transport tests. Python version shipped with CentOS6 is known to
    * have an older version of python resulting in test failures.
    */
-  private boolean pythonSupportsSSLContext() throws Exception {
+  protected boolean pythonSupportsSSLContext() throws Exception {
     // Runs the following command:
     // python -c "import ssl; print hasattr(ssl, 'create_default_context')"
     String[] cmd = {
diff --git a/fe/src/test/java/org/apache/impala/customcluster/LdapKerberosImpalaShellTest.java b/fe/src/test/java/org/apache/impala/customcluster/LdapKerberosImpalaShellTest.java
new file mode 100644
index 000000000..2f50098ac
--- /dev/null
+++ b/fe/src/test/java/org/apache/impala/customcluster/LdapKerberosImpalaShellTest.java
@@ -0,0 +1,640 @@
+// 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.impala.customcluster;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.directory.server.core.annotations.ApplyLdifFiles;
+import org.apache.directory.server.core.annotations.CreateDS;
+import org.apache.directory.server.core.annotations.CreatePartition;
+import org.junit.Assume;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.impala.testutil.LdapUtil.GROUP_DN_PATTERN;
+import static org.apache.impala.testutil.LdapUtil.TEST_PASSWORD_1;
+import static org.apache.impala.testutil.LdapUtil.TEST_USER_1;
+import static org.apache.impala.testutil.LdapUtil.TEST_USER_2;
+import static org.apache.impala.testutil.LdapUtil.TEST_USER_3;
+import static org.apache.impala.testutil.LdapUtil.TEST_USER_4;
+import static org.apache.impala.testutil.LdapUtil.TEST_USER_7;
+import static org.apache.impala.testutil.LdapUtil.TEST_USER_DN_1;
+import static org.apache.impala.testutil.LdapUtil.TEST_USER_GROUP;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Impala shell connectivity tests with Kerberos authentication.
+ */
+@CreateDS(name = "myDS",
+        partitions = { @CreatePartition(name = "test", suffix = "dc=myorg,dc=com") })
+@ApplyLdifFiles({"users.ldif"})
+public class LdapKerberosImpalaShellTest extends LdapKerberosImpalaShellTestBase {
+
+  /**
+   * Tests Kerberos authentication with custom LDAP user and group filter configs
+   * with search bind enabled and group filter check disabled.
+   */
+  @Test
+  public void testShellKerberosAuthWithCustomLdapFiltersAndSearchBindNoGroupFilterCheck()
+          throws Exception {
+    Map<String, String> flags = mergeFlags(
+
+            // enable Kerberos authentication
+            kerberosKdcEnvironment.getKerberosAuthFlags(),
+
+            // enable LDAP authentication with search bind
+            getLdapSearchBindFlags(),
+
+            // allow using custom filters with Kerberos authentication
+            ImmutableMap.of(
+                    "allow_custom_ldap_filters_with_kerberos_auth", "true"
+            ),
+
+            // define custom LDAP filters corresponding to the values
+            // in fe/src/test/resources/users.ldif
+            getCustomLdapFilterFlags()
+    );
+    int ret = startImpalaCluster(flagsToArgs(flags));
+    assertEquals(ret, 0);
+
+    // Cluster should pass impala-shell Kerberos auth tests executed with a user
+    // that does not exist in LDAP.
+    testShellKerberosAuthWithUser( kerberosKdcEnvironment, "user",
+            /* shouldSucceed */ true);
+  }
+
+  /**
+   * Tests Kerberos authentication with LDAP authentication and search bind enabled,
+   * custom LDAP user and group filter configs provided,
+   * and group filter check with Kerberos auth enabled.
+   */
+  @Test
+  public void testShellKerberosAuthWithCustomLdapFiltersAndSearchBindGroupFilterCheck()
+          throws Exception {
+    Map<String, String> flags = mergeFlags(
+
+            // enable Kerberos authentication
+            kerberosKdcEnvironment.getKerberosAuthFlags(),
+
+            // enable LDAP authentication with search bind
+            getLdapSearchBindFlags(),
+
+            // allow using custom filters with Kerberos authentication
+            ImmutableMap.of(
+                    "allow_custom_ldap_filters_with_kerberos_auth", "true"
+            ),
+
+            // define custom LDAP filters corresponding to the values
+            // in fe/src/test/resources/users.ldif
+            getCustomLdapFilterFlags(),
+
+            // allow using custom filters with Kerberos authentication
+            ImmutableMap.of(
+                    "enable_group_filter_check_for_authenticated_kerberos_user", "true"
+            )
+    );
+    int ret = startImpalaCluster(flagsToArgs(flags));
+    assertEquals(ret, 0);
+
+    // Group filter check is enabled with Kerberos auth:
+    // Cluster should pass impala-shell Kerberos auth tests executed
+    // with a user that exists in LDAP and passes LDAP group filter check.
+    testShellKerberosAuthWithUser(kerberosKdcEnvironment, TEST_USER_1,
+            /* shouldSucceed */ true);
+
+    // Kerberos authentication should fail with impala-shell with a user
+    // that exists in LDAP but it does not pass LDAP group filter check.
+    testShellKerberosAuthWithUser(kerberosKdcEnvironment, TEST_USER_2,
+            /* shouldSucceed */ false);
+    testShellKerberosAuthWithUser(kerberosKdcEnvironment, TEST_USER_3,
+            /* shouldSucceed */ false);
+
+    // Kerberos authentication should also fail with a user
+    // that does not exist in LDAP.
+    testShellKerberosAuthWithUser(kerberosKdcEnvironment,"user",
+            /* shouldSucceed */ false);
+  }
+
+  /**
+   * Tests user impersonation with Kerberos authentication over the HTTP protocol
+   * with LDAP authentication and LDAP search bind enabled, custom LDAP user filter
+   * config provided.
+   */
+  @Test
+  public void testHttpImpersonationWithKerberosAuthAndLdapSearchBind() throws Exception {
+    // Ignore the test if python SSLContext support is not available.
+    Assume.assumeTrue(pythonSupportsSSLContext());
+
+    String ldapUri = String.format("ldap://localhost:%s",
+            serverRule.getLdapServer().getPort());
+    String passwordCommand = String.format("'echo -n %s'", TEST_PASSWORD_1);
+
+    Map<String, String> flags = mergeFlags(
+
+            // enable Kerberos authentication
+            kerberosKdcEnvironment.getKerberosAuthFlags(),
+
+            // enable LDAP authentication with search bind,
+            // define custom LDAP user filter corresponding to the values
+            // in fe/src/test/resources/users.ldif,
+            // and allow using custom filters with Kerberos authentication
+            ImmutableMap.of(
+                    "enable_ldap_auth", "true",
+                    "ldap_uri", ldapUri,
+                    "ldap_passwords_in_clear_ok", "true",
+                    "ldap_user_search_basedn", defaultUserSearchBaseDn,
+                    "ldap_user_filter", "(cn={0})",
+                    "ldap_search_bind_authentication", "true",
+                    "ldap_bind_dn", TEST_USER_DN_1,
+                    "ldap_bind_password_cmd", passwordCommand,
+                    "allow_custom_ldap_filters_with_kerberos_auth", "true"
+            ),
+
+            // set proxy user: allow TEST_USER_1 to act as a proxy for delegateUser_
+            ImmutableMap.of(
+                    "authorized_proxy_user_config",
+                    String.format("%s=%s", TEST_USER_1, delegateUser_)
+            )
+    );
+    int ret = startImpalaCluster(flagsToArgs(flags));
+    assertEquals(ret, 0);
+
+    String errTemplate = "User '%s' is not authorized to delegate to '%s'";
+
+    // Run with an invalid proxy user.
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_2,
+            "/?doAs=" + delegateUser_,
+            /* shouldSucceed */ false, "",
+            String.format(errTemplate,
+                    kerberosKdcEnvironment.getUserPrincipal(TEST_USER_2),
+                    delegateUser_));
+
+    // Run with a valid proxy user but invalid delegate user.
+    String invalidDelegateUser = "invalid-delegate-user";
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_1,
+            "/?doAs=" + invalidDelegateUser,
+            /* shouldSucceed */ false, "",
+            String.format(errTemplate,
+                    kerberosKdcEnvironment.getUserPrincipal(TEST_USER_1),
+                    invalidDelegateUser));
+
+    // 'doAs' parameter that cannot be decoded.
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_1,
+            "/?doAs=%",
+            /* shouldSucceed */ false,
+            "", "Not connected to Impala");
+
+    // Successfully delegate.
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_1,
+            "/?doAs=" + delegateUser_,
+            /* shouldSucceed */ true,
+            delegateUser_, "");
+  }
+
+  /**
+   * Tests user impersonation with Kerberos authentication over the HTTP protocol
+   * with LDAP authentication and LDAP search bind enabled, custom LDAP user and
+   * group filter config provided.
+   * This test validates that LDAP filters applied to delegate users regardless
+   * of the value of enable_group_filter_check_for_authenticated_kerberos_user flag.
+   */
+  @Test
+  public void testHttpImpersonationWithKerberosAuthAndLdapSearchBindWithGroupFilters()
+          throws Exception {
+    // Ignore the test if python SSLContext support is not available.
+    Assume.assumeTrue(pythonSupportsSSLContext());
+
+    Map<String, String> flags = mergeFlags(
+
+            // enable Kerberos authentication
+            kerberosKdcEnvironment.getKerberosAuthFlags(),
+
+            // enable LDAP authentication with search bind
+            getLdapSearchBindFlags(),
+
+            // allow using custom filters with Kerberos authentication
+            ImmutableMap.of(
+                    "allow_custom_ldap_filters_with_kerberos_auth", "true"
+            ),
+
+            // define custom LDAP filters corresponding to the values
+            // in fe/src/test/resources/users.ldif
+            getCustomLdapFilterFlags(),
+
+            // set proxy user: allow TEST_USER_1 to act as a proxy user for
+            // any other user
+            ImmutableMap.of(
+                    "authorized_proxy_user_config", String.format("%s=*", TEST_USER_1)
+            )
+
+            // enable_group_filter_check_for_authenticated_kerberos_user flag
+            // not defined
+    );
+    int ret = startImpalaCluster(flagsToArgs(flags));
+    assertEquals(ret, 0);
+
+    // Kerberos authentication should fail with a delegate user
+    // that does not exist in LDAP.
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_1,
+            "/?doAs=" + delegateUser_,
+            /* shouldSucceed */ false,
+            "", "User is not authorized.");
+
+    // Kerberos authentication should fail with delegate users that exist in LDAP,
+    // but do not pass LDAP filter checks.
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_1,
+            "/?doAs=" + TEST_USER_2,
+            /* shouldSucceed */ false,
+            "", "User is not authorized.");
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_1,
+            "/?doAs=" + TEST_USER_3,
+            /* shouldSucceed */ false,
+            "", "User is not authorized.");
+
+    // Kerberos authentication should succeed with a delegate user
+    // that exists in LDAP and passes LDAP filter checks.
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_1,
+            "/?doAs=" + TEST_USER_7,
+            /* shouldSucceed */ true,
+            TEST_USER_7, "");
+  }
+
+  /**
+   * Tests the interaction between LDAP user and group filter configs and proxy user
+   * configs when authenticating with Kerberos and LDAP authentication is also enabled,
+   * LDAP search bind configured.
+   */
+  @Test
+  public void testLdapFiltersWithProxyWithKerberosAuthAndLdapSearchBind()
+          throws Exception {
+
+    String customLdapUserFilter =
+            String.format("(&(objectClass=person)(cn={0})(!(cn=%s)))", TEST_USER_2);
+    String customLdapGroupFilter = "(&(cn=group1)(uniqueMember={0}))";
+
+    Map<String, String> flags = mergeFlags(
+
+            // enable Kerberos authentication
+            kerberosKdcEnvironment.getKerberosAuthFlags(),
+
+            // enable LDAP authentication with search bind
+            getLdapSearchBindFlags(),
+
+            // allow using custom filters with Kerberos authentication
+            ImmutableMap.of(
+                    "allow_custom_ldap_filters_with_kerberos_auth", "true"
+            ),
+
+            // define custom LDAP filters corresponding to the values
+            // in fe/src/test/resources/users.ldif
+            ImmutableMap.of(
+                    "ldap_user_filter", customLdapUserFilter,
+                    "ldap_group_filter", customLdapGroupFilter
+            ),
+
+            // set proxy user:
+            // allow TEST_USER_4 to act as a proxy for any other user
+            ImmutableMap.of(
+                    "authorized_proxy_user_config",
+                    String.format("%s=*", TEST_USER_4)
+            )
+    );
+    int ret = startImpalaCluster(flagsToArgs(flags));
+    assertEquals(ret, 0);
+
+
+    // Run as the proxy user with a delegate that passes both filters, should succeed
+    // and return the delegate user's name.
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_4,
+            "/?doAs=" + TEST_USER_1,
+            /* shouldSucceed */ true,
+            TEST_USER_1, "");
+
+    // Run as the proxy user with a delegate that only passes the user filter, should
+    // fail.
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_4,
+            "/?doAs=" + TEST_USER_3,
+            /* shouldSucceed */ false,
+            "", "Not connected to Impala");
+
+    // Run as the proxy user with a delegate that only passes the group filter, should
+    // fail.
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_4,
+            "/?doAs=" + TEST_USER_2,
+            /* shouldSucceed */ false,
+            "", "Not connected to Impala");
+
+    // Run as the proxy with a delegate that doesn't pass either filter, should fail.
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_4,
+            "/?doAs=" + TEST_USER_4,
+            /* shouldSucceed */ false,
+            "", "Not connected to Impala");
+
+    // Run as the proxy without a delegate user, should fail since the proxy user won't
+    // pass the filters.
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_4,
+            "/",
+            /* shouldSucceed */ false,
+            "", "");
+  }
+
+  /**
+   * Tests user impersonation with Kerberos authentication over the HTTP protocol
+   * with LDAP authentication and LDAP simple bind enabled.
+   */
+  @Test
+  public void testHttpImpersonationWithKerberosAuthAndLdapSimpleBind() throws Exception {
+    // Ignore the test if python SSLContext support is not available.
+    Assume.assumeTrue(pythonSupportsSSLContext());
+
+    String ldapUri = String.format("ldap://localhost:%s",
+            serverRule.getLdapServer().getPort());
+
+    Map<String, String> flags = mergeFlags(
+
+            // enable Kerberos authentication
+            kerberosKdcEnvironment.getKerberosAuthFlags(),
+
+            // enable LDAP authentication with simple bind
+            ImmutableMap.of(
+                    "enable_ldap_auth", "true",
+                    "ldap_uri", ldapUri,
+                    "ldap_passwords_in_clear_ok", "true",
+                    "ldap_bind_pattern","'cn=#UID,ou=Users,dc=myorg,dc=com'"
+            ),
+
+            // set proxy user: allow TEST_USER_1 to act as a proxy for delegateUser_
+            ImmutableMap.of(
+                    "authorized_proxy_user_config",
+                    String.format("%s=%s", TEST_USER_1, delegateUser_)
+            )
+    );
+    int ret = startImpalaCluster(flagsToArgs(flags));
+    assertEquals(ret, 0);
+
+    String errTemplate = "User '%s' is not authorized to delegate to '%s'";
+
+    // Run with an invalid proxy user.
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_2,
+            "/?doAs=" + delegateUser_,
+            /* shouldSucceed */ false,
+            "",
+            String.format(errTemplate,
+                    kerberosKdcEnvironment.getUserPrincipal(TEST_USER_2),
+                    delegateUser_));
+
+    // Run with a valid proxy user but invalid delegate user.
+    String invalidDelegateUser = "invalid-delegate-user";
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_1,
+            "/?doAs=" + invalidDelegateUser,
+            /* shouldSucceed */ false,
+            "",
+            String.format(errTemplate,
+                    kerberosKdcEnvironment.getUserPrincipal(TEST_USER_1),
+                    invalidDelegateUser));
+
+    // 'doAs' parameter that cannot be decoded.
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_1,
+            "/?doAs=%",
+            /* shouldSucceed */ false,
+            "", "Not connected to Impala");
+
+    // Successfully delegate.
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_1,
+            "/?doAs=" + delegateUser_,
+            /* shouldSucceed */ true,
+            delegateUser_, "");
+  }
+
+  /**
+   * Tests Kerberos authentication with custom LDAP user and group filter configs
+   * with simple bind enabled and group filter check disabled.
+   */
+  @Test
+  public void testShellKerberosAuthWithCustomLdapFiltersAndSimpleBindNoGroupFilterCheck()
+          throws Exception {
+    Map<String, String> flags = mergeFlags(
+
+            // enable Kerberos authentication
+            kerberosKdcEnvironment.getKerberosAuthFlags(),
+
+            // enable LDAP authentication with simple bind
+            getLdapSimpleBindFlags(),
+
+            // allow using custom filters with Kerberos authentication
+            ImmutableMap.of(
+                    "allow_custom_ldap_filters_with_kerberos_auth", "true"
+            ),
+
+            // set custom user and group filters
+            getCustomLdapSimpleBindSearchFilterFlags()
+    );
+    int ret = startImpalaCluster(flagsToArgs(flags));
+    assertEquals(ret, 0);
+
+    // Cluster should pass impala-shell Kerberos auth tests executed with a user
+    // that does not exist in LDAP.
+    testShellKerberosAuthWithUser( kerberosKdcEnvironment, "user",
+            /* shouldSucceed */ true);
+  }
+
+  /**
+   * Tests Kerberos authentication with custom LDAP user and group filter configs
+   * with simple bind and group filter check enabled.
+   */
+  @Test
+  public void testShellKerberosAuthWithCustomLdapFiltersAndSimpleBindGroupFilterCheck()
+          throws Exception {
+    Map<String, String> flags = mergeFlags(
+
+            // enable Kerberos authentication
+            kerberosKdcEnvironment.getKerberosAuthFlags(),
+
+            // enable LDAP authentication with simple bind
+            getLdapSimpleBindFlags(),
+
+            // allow using custom filters with Kerberos authentication
+            ImmutableMap.of(
+                    "allow_custom_ldap_filters_with_kerberos_auth", "true"
+            ),
+
+            // set custom user and group filters
+            getCustomLdapSimpleBindSearchFilterFlags(),
+
+            // allow using custom filters with Kerberos authentication
+            ImmutableMap.of(
+                    "enable_group_filter_check_for_authenticated_kerberos_user", "true"
+            )
+    );
+    int ret = startImpalaCluster(flagsToArgs(flags));
+    assertEquals(ret, 0);
+
+    // Group filter check is enabled with Kerberos auth:
+    // Cluster should pass impala-shell Kerberos auth tests executed
+    // with a user that exists in LDAP and passes LDAP group filter check.
+    testShellKerberosAuthWithUser(kerberosKdcEnvironment, TEST_USER_1,
+            /* shouldSucceed */ true);
+
+    // Kerberos authentication should fail with impala-shell with a user
+    // that exists in LDAP but it does not pass LDAP group filter check.
+    testShellKerberosAuthWithUser(kerberosKdcEnvironment, TEST_USER_2,
+            /* shouldSucceed */ false);
+    testShellKerberosAuthWithUser(kerberosKdcEnvironment, TEST_USER_3,
+            /* shouldSucceed */ false);
+
+    // Kerberos authentication should also fail with a user
+    // that does not exist in LDAP.
+    testShellKerberosAuthWithUser(kerberosKdcEnvironment,"user",
+            /* shouldSucceed */ false);
+  }
+
+  /**
+   * Tests the interaction between LDAP user and group filter configs and proxy user
+   * configs when authenticating with Kerberos and LDAP authentication is also enabled,
+   * LDAP simple bind configured.
+   */
+  @Test
+  public void testLdapFiltersWithProxyWithKerberosAuthAndLdapSimpleBind()
+          throws Exception {
+    // 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.
+    Map<String, String> flags = mergeFlags(
+
+            // enable Kerberos authentication
+            kerberosKdcEnvironment.getKerberosAuthFlags(),
+
+            // enable LDAP authentication with simple bind
+            getLdapSimpleBindFlags(),
+
+            // allow using custom filters with Kerberos authentication
+            ImmutableMap.of(
+                    "allow_custom_ldap_filters_with_kerberos_auth", "true"
+            ),
+
+            // set custom user and group filters
+            getCustomLdapSimpleBindSearchFilterFlags(),
+
+            // set proxy user:
+            // allow TEST_USER_4 to act as a proxy for any other user
+            ImmutableMap.of(
+                    "authorized_proxy_user_config",
+                    String.format("%s=*", TEST_USER_4)
+            )
+    );
+    int ret = startImpalaCluster(flagsToArgs(flags));
+    assertEquals(ret, 0);
+
+    // Run as the proxy user with a delegate that passes both filters, should succeed
+    // and return the delegate user's name.
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_4,
+            "/?doAs=" + TEST_USER_1,
+            /* shouldSucceed */ true,
+            TEST_USER_1, "");
+
+    // Run as the proxy user with a delegate that only passes the user filter, should
+    // fail.
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_4,
+            "/?doAs=" + TEST_USER_3,
+            /* shouldSucceed */false,
+            "","Not connected to Impala");
+
+    // Run as the proxy user with a delegate that only passes the group filter, should
+    // fail.
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_4,
+            "/?doAs=" + TEST_USER_2, /* shouldSucceed */ false,
+            "", "Not connected to Impala");
+
+    // Run as the proxy with a delegate that doesn't pass either filter, should fail.
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_4,
+            "/?doAs=" + TEST_USER_4, /* shouldSucceed */ false,
+            "", "Not connected to Impala");
+
+    // Run as the proxy without a delegate user, should fail since the proxy user won't
+    // pass the filters.
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_4,
+            "/", /* shouldSucceed */ false,
+            "", "");
+  }
+
+  /**
+   * Tests Kerberos authentication using impala-shell.
+   */
+  protected void testShellKerberosAuthWithUser(
+          KerberosKdcEnvironment kerberosKdcEnvironment, String username,
+          boolean shouldSucceed) throws Exception {
+
+    List<String> protocolsToTest = Arrays.asList("beeswax", "hs2");
+    if (pythonSupportsSSLContext()) {
+      // http transport tests will fail with older python versions (IMPALA-8873)
+      protocolsToTest = Arrays.asList("beeswax", "hs2", "hs2-http");
+    }
+
+    // create user principal in KDC
+    // and create a credentials cache file with a valid TGT ticket
+    String credentialsCacheFilePath =
+            kerberosKdcEnvironment.createUserPrincipalAndCredentialsCache(username);
+
+    for (String protocol : protocolsToTest) {
+      String[] command = {
+              "impala-shell.sh",
+              String.format("--protocol=%s", protocol),
+              "--kerberos",
+              "--query=select logged_in_user()"
+      };
+      String expectedOut =
+              shouldSucceed ? kerberosKdcEnvironment.getUserPrincipal(username) : "";
+      String expectedErr =
+              shouldSucceed ? "Starting Impala Shell with Kerberos authentication" :
+                      "Not connected to Impala";
+      RunShellCommand.Run(command,
+              kerberosKdcEnvironment.getImpalaShellEnv(credentialsCacheFilePath),
+              shouldSucceed, expectedOut, expectedErr);
+    }
+  }
+
+  private Map<String, String> getLdapSimpleBindFlags() {
+    String ldapUri = String.format("ldap://localhost:%s",
+            serverRule.getLdapServer().getPort());
+    String passwordCommand = String.format("'echo -n %s'", TEST_PASSWORD_1);
+    return ImmutableMap.of(
+            "enable_ldap_auth", "true",
+            "ldap_uri", ldapUri,
+            "ldap_passwords_in_clear_ok", "true",
+            "ldap_bind_pattern","'cn=#UID,ou=Users,dc=myorg,dc=com'",
+            "ldap_group_dn_pattern", GROUP_DN_PATTERN,
+            "ldap_group_membership_key", "uniqueMember",
+            "ldap_group_class_key", "groupOfUniqueNames",
+            "ldap_bind_dn", TEST_USER_DN_1,
+            "ldap_bind_password_cmd", passwordCommand
+    );
+  }
+
+  private Map<String, String> getCustomLdapSimpleBindSearchFilterFlags() {
+    String customGroupFilter = String.format("%s,another-group", TEST_USER_GROUP);
+    String customUserFilter =  String.format("%s,%s,another-user", TEST_USER_1,
+            TEST_USER_3);
+    return ImmutableMap.of(
+            "ldap_group_filter", customGroupFilter,
+            "ldap_user_filter", customUserFilter
+    );
+  }
+
+}
diff --git a/fe/src/test/java/org/apache/impala/customcluster/LdapKerberosImpalaShellTestBase.java b/fe/src/test/java/org/apache/impala/customcluster/LdapKerberosImpalaShellTestBase.java
new file mode 100644
index 000000000..5a857df39
--- /dev/null
+++ b/fe/src/test/java/org/apache/impala/customcluster/LdapKerberosImpalaShellTestBase.java
@@ -0,0 +1,123 @@
+// 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.impala.customcluster;
+
+import com.google.common.collect.ImmutableMap;
+import org.junit.ClassRule;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+import static org.apache.impala.testutil.LdapUtil.TEST_PASSWORD_1;
+import static org.apache.impala.testutil.LdapUtil.TEST_USER_2;
+import static org.apache.impala.testutil.LdapUtil.TEST_USER_DN_1;
+
+/**
+ * Base class for Impala shell connectivity tests with LDAP and Kerberos authentication.
+ */
+public class LdapKerberosImpalaShellTestBase extends LdapImpalaShellTest {
+
+  protected final static String defaultUserSearchBaseDn = "dc=myorg,dc=com";
+  protected final static String defaultGroupSearchBaseDn = "ou=Groups,dc=myorg,dc=com";
+
+  @ClassRule
+  public static KerberosKdcEnvironment kerberosKdcEnvironment =
+          new KerberosKdcEnvironment(new TemporaryFolder());
+
+  protected Map<String, String> getLdapSearchBindFlags() {
+    return getLdapSearchBindFlags(defaultUserSearchBaseDn, defaultGroupSearchBaseDn);
+  }
+
+  protected Map<String, String> getLdapSearchBindFlags(
+          String userSearchBaseDn, String groupSearchBaseDn) {
+    String ldapUri = String.format("ldap://localhost:%s",
+            serverRule.getLdapServer().getPort());
+    String passwordCommand = String.format("'echo -n %s'", TEST_PASSWORD_1);
+    return ImmutableMap.of(
+            "enable_ldap_auth", "true",
+            "ldap_uri", ldapUri,
+            "ldap_passwords_in_clear_ok", "true",
+            "ldap_user_search_basedn", userSearchBaseDn,
+            "ldap_group_search_basedn", groupSearchBaseDn,
+            "ldap_search_bind_authentication", "true",
+            "ldap_bind_dn", TEST_USER_DN_1,
+            "ldap_bind_password_cmd", passwordCommand
+    );
+  }
+
+  protected Map<String, String> getCustomLdapFilterFlags() {
+    String customLdapUserFilter =
+            String.format("(&(objectClass=person)(cn={0})(!(cn=%s)))", TEST_USER_2);
+    String customLdapGroupFilter = "(uniqueMember={0})";
+    return ImmutableMap.of(
+            "ldap_user_filter", customLdapUserFilter,
+            "ldap_group_filter", customLdapGroupFilter
+    );
+  }
+
+  @Override
+  protected int startImpalaCluster(String args) throws IOException, InterruptedException {
+    return kerberosKdcEnvironment.startImpalaClusterWithArgs(args);
+  }
+
+  public static String flagsToArgs(Map<String, String> flags) {
+    return flags.entrySet().stream()
+            .map(entry -> "--" + entry.getKey() + "=" + entry.getValue() + " ")
+            .collect(Collectors.joining());
+  }
+
+  @SafeVarargs
+  public static Map<String, String> mergeFlags(Map<String, String>... flags) {
+    return Arrays.stream(flags)
+            .filter(Objects::nonNull)
+            .flatMap(map -> map.entrySet().stream())
+            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+  }
+
+  /**
+   * Tests Kerberos authentication using impala-shell over HTTP protocol,
+   * using the HTTP path to specify the 'doAs' parameter.
+   */
+  protected void testShellKerberosAuthWithUserWithHttpPath(
+          KerberosKdcEnvironment kerberosKdcEnvironment, String username,
+          String httpPath, boolean shouldSucceed, String expectedOut,
+          String expectedErr) throws Exception {
+
+    // Run with an invalid proxy user.
+    String[] command = {
+            "impala-shell.sh",
+            "--protocol=hs2-http",
+            "--kerberos",
+            "--query=select logged_in_user()",
+            "--http_path=" + httpPath};
+
+    // create user principal in KDC
+    // and create a credentials cache file with a valid TGT ticket
+    String credentialsCacheFilePath =
+            kerberosKdcEnvironment.createUserPrincipalAndCredentialsCache(username);
+
+    RunShellCommand.Run(command,
+            kerberosKdcEnvironment.getImpalaShellEnv(credentialsCacheFilePath),
+            shouldSucceed, expectedOut, expectedErr);
+  }
+
+}
diff --git a/fe/src/test/java/org/apache/impala/customcluster/LdapSearchBindDefaultFiltersKerberosImpalaShellTest.java b/fe/src/test/java/org/apache/impala/customcluster/LdapSearchBindDefaultFiltersKerberosImpalaShellTest.java
new file mode 100644
index 000000000..398402328
--- /dev/null
+++ b/fe/src/test/java/org/apache/impala/customcluster/LdapSearchBindDefaultFiltersKerberosImpalaShellTest.java
@@ -0,0 +1,165 @@
+// 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.impala.customcluster;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.directory.server.core.annotations.ApplyLdifFiles;
+import org.apache.directory.server.core.annotations.CreateDS;
+import org.apache.directory.server.core.annotations.CreatePartition;
+import org.junit.Test;
+
+import java.util.Map;
+
+import static org.apache.impala.testutil.LdapUtil.TEST_USER_1;
+import static org.apache.impala.testutil.LdapUtil.TEST_USER_2;
+import static org.apache.impala.testutil.LdapUtil.TEST_USER_3;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Impala shell connectivity tests with LDAP Search Bind and Kerberos authentication.
+ *
+ * These test are required to validate backwards compatibility with the changes
+ * introduced in IMPALA-11726. Neither the allow_custom_ldap_filters_with_kerberos_auth,
+ * nor the enable_group_filter_check_for_authenticated_kerberos_user flag is set in
+ * these tests.
+ */
+@CreateDS(name = "myAD",
+        partitions = {@CreatePartition(name = "test", suffix = "dc=myorg,dc=com")})
+@ApplyLdifFiles({"adschema.ldif", "adusers.ldif"})
+public class LdapSearchBindDefaultFiltersKerberosImpalaShellTest
+        extends LdapKerberosImpalaShellTestBase {
+
+  /**
+   * Tests custom LDAP user and group filter configs with Search Bind and
+   * Kerberos authentication enabled.
+   *
+   * With custom LDAP filters without Kerberos authentication the cluster
+   * should start successfully, but in this test the Kerberos authentication
+   * is enabled with the principal flag, so that the cluster should not start up.
+   */
+  @Test
+  public void testCustomLdapFiltersNotAllowedWithKerberos() throws Exception {
+    Map<String, String> flags = mergeFlags(
+
+            // enable Kerberos authentication
+            kerberosKdcEnvironment.getKerberosAuthFlags(),
+
+            // enable LDAP authentication with search bind
+            getLdapSearchBindFlags(),
+
+            // define custom LDAP filters
+            getCustomLdapFilterFlags(),
+
+            // to prevent the test from being considered unstable, we should
+            // disable minidump creation
+            ImmutableMap.of(
+                    "enable_minidumps", "false"
+            )
+    );
+
+    int ret = startImpalaCluster(flagsToArgs(flags));
+    assertEquals(ret, 1); // cluster should not start up
+  }
+
+  /**
+   * Tests default LDAP user and group filters with Search Bind and
+   * Kerberos authentication enabled.
+   *
+   * Without custom LDAP filters the LDAP search bind and the Kerberos authentication
+   * should work together.
+   * This test uses an AD-like LDAP scheme that matches the default LDAP filters.
+   */
+  @Test
+  public void testDefaultLdapFiltersAreAllowedWithSearchBindAndKerberos()
+          throws Exception {
+    String userSearchBaseDn = "ou=Users,dc=myorg,dc=com";
+    Map<String, String> flags = mergeFlags(
+
+            // enable Kerberos authentication
+            kerberosKdcEnvironment.getKerberosAuthFlags(),
+
+            // enable LDAP authentication with search bind
+            // this test setup requires different user search base dn
+            getLdapSearchBindFlags(userSearchBaseDn, defaultGroupSearchBaseDn)
+
+            // custom LDAP filters not defined
+    );
+    int ret = startImpalaCluster(flagsToArgs(flags));
+    assertEquals(ret, 0); // cluster should start up
+
+    testLdapFiltersImpl(); // cluster should pass LDAP filter tests with default filters
+  }
+
+  /**
+   * Tests default LDAP user and group filters are applied to delegate users when
+   * Search Bind and Kerberos authentication enabled and the proxy user authenticates
+   * with Kerberos.
+   *
+   * This test uses an AD-like LDAP scheme that matches the default LDAP filters.
+   */
+  @Test
+  public void testDefaultLdapFiltersAreAppliedToDelegateUserWithKerberosAuth()
+          throws Exception {
+    String userSearchBaseDn = "ou=Users,dc=myorg,dc=com";
+    Map<String, String> flags = mergeFlags(
+
+            // enable Kerberos authentication
+            kerberosKdcEnvironment.getKerberosAuthFlags(),
+
+            // enable LDAP authentication with search bind
+            // this test setup requires different user search base dn
+            getLdapSearchBindFlags(userSearchBaseDn, defaultGroupSearchBaseDn),
+
+            // custom LDAP filters not defined
+
+            // set proxy user: allow TEST_USER_1 to act as a proxy user for
+            // any other user
+            ImmutableMap.of(
+                    "authorized_proxy_user_config", String.format("%s=*", TEST_USER_1)
+            )
+    );
+    int ret = startImpalaCluster(flagsToArgs(flags));
+    assertEquals(ret, 0); // cluster should start up
+
+    // Kerberos authentication should fail with a delegate user
+    // that does not exist in LDAP.
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_1,
+            "/?doAs=" + delegateUser_,
+            /* shouldSucceed */ false,
+            "", "User is not authorized.");
+
+    // Kerberos authentication should fail with delegate users that exist in LDAP,
+    // but do not pass LDAP filter checks.
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_1,
+            "/?doAs=" + TEST_USER_2,
+            /* shouldSucceed */ false,
+            "", "User is not authorized.");
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_1,
+            "/?doAs=" + TEST_USER_3,
+            /* shouldSucceed */ false,
+            "", "User is not authorized.");
+
+    // Kerberos authentication should succeed with a delegate user
+    // that exists in LDAP and passes LDAP filter checks.
+    testShellKerberosAuthWithUserWithHttpPath(kerberosKdcEnvironment, TEST_USER_1,
+            "/?doAs=" + TEST_USER_1,
+            /* shouldSucceed */ true,
+            TEST_USER_1, "");
+  }
+
+}
diff --git a/fe/src/test/java/org/apache/impala/customcluster/LdapSearchBindImpalaShellTest.java b/fe/src/test/java/org/apache/impala/customcluster/LdapSearchBindImpalaShellTest.java
index 38d74540c..4d76471e4 100644
--- a/fe/src/test/java/org/apache/impala/customcluster/LdapSearchBindImpalaShellTest.java
+++ b/fe/src/test/java/org/apache/impala/customcluster/LdapSearchBindImpalaShellTest.java
@@ -17,26 +17,79 @@
 
 package org.apache.impala.customcluster;
 
+import static org.apache.impala.customcluster.LdapKerberosImpalaShellTestBase.flagsToArgs;
+import static org.apache.impala.customcluster.LdapKerberosImpalaShellTestBase.mergeFlags;
 import static org.apache.impala.testutil.LdapUtil.*;
 
+import com.google.common.collect.ImmutableMap;
 import org.apache.directory.server.core.annotations.CreateDS;
 import org.apache.directory.server.core.annotations.CreatePartition;
+import org.junit.ClassRule;
 import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
 
 /**
  * Impala shell connectivity tests with Search Bind LDAP authentication.
+ *
+ * The test suite is parameterized, all tests are executed with both Kerberos
+ * authentication disabled and with Kerberos authentication enabled to validate
+ * that LDAP search bind authentication is not broken even if Kerberos authentication
+ * is enabled.
  */
 @CreateDS(name = "myDS",
     partitions = { @CreatePartition(name = "test", suffix = "dc=myorg,dc=com") })
+@RunWith(Parameterized.class)
 public class LdapSearchBindImpalaShellTest extends LdapImpalaShellTest {
+
+  @ClassRule
+  public static KerberosKdcEnvironment kerberosKdcEnvironment =
+          new KerberosKdcEnvironment(new TemporaryFolder());
+
+  private final boolean kerberosAuthenticationEnabled;
+
+  @Parameterized.Parameters(name = "kerberosAuthenticationEnabled={0}")
+  public static Boolean[] kerberosAuthenticationEnabled() {
+    return new Boolean[] {Boolean.FALSE, Boolean.TRUE};
+  }
+
+  public LdapSearchBindImpalaShellTest(boolean isKerberosAuthenticationEnabled) {
+    this.kerberosAuthenticationEnabled = isKerberosAuthenticationEnabled;
+  }
+
+
+  @Override
+  protected int startImpalaCluster(String args) throws IOException, InterruptedException {
+    if (kerberosAuthenticationEnabled) {
+      return kerberosKdcEnvironment.startImpalaClusterWithArgs(args);
+    } else {
+      return super.startImpalaCluster(args);
+    }
+  }
+
   @Override
   public void setUp(String extraArgs) throws Exception {
     String searchBindArgs = String.format("--ldap_search_bind_authentication=true "
-            + "--ldap_bind_dn=%s --ldap_bind_password_cmd='echo -n %s' %s",
-        TEST_USER_DN_1, TEST_PASSWORD_1, extraArgs);
+                    + "--ldap_bind_dn=%s --ldap_bind_password_cmd='echo -n %s' %s %s",
+            TEST_USER_DN_1, TEST_PASSWORD_1, getKerberosArgs(), extraArgs);
     super.setUp(searchBindArgs);
   }
 
+  private String getKerberosArgs() throws IOException {
+    return kerberosAuthenticationEnabled ?
+            flagsToArgs(mergeFlags(
+                    kerberosKdcEnvironment.getKerberosAuthFlags(),
+                    ImmutableMap.of(
+                            "allow_custom_ldap_filters_with_kerberos_auth", "true"
+                    )
+            ))
+            :
+            "";  // empty if Kerberos authentication is disabled
+  }
+
   /**
    * Tests ldap authentication using impala-shell.
    */
diff --git a/fe/src/test/java/org/apache/impala/customcluster/LdapSimpleBindImpalaShellTest.java b/fe/src/test/java/org/apache/impala/customcluster/LdapSimpleBindImpalaShellTest.java
index 1ac648faa..a8f0255ac 100644
--- a/fe/src/test/java/org/apache/impala/customcluster/LdapSimpleBindImpalaShellTest.java
+++ b/fe/src/test/java/org/apache/impala/customcluster/LdapSimpleBindImpalaShellTest.java
@@ -17,25 +17,78 @@
 
 package org.apache.impala.customcluster;
 
+import static org.apache.impala.customcluster.LdapKerberosImpalaShellTestBase.flagsToArgs;
+import static org.apache.impala.customcluster.LdapKerberosImpalaShellTestBase.mergeFlags;
 import static org.apache.impala.testutil.LdapUtil.*;
 
+import com.google.common.collect.ImmutableMap;
 import org.apache.directory.server.core.annotations.CreateDS;
 import org.apache.directory.server.core.annotations.CreatePartition;
+import org.junit.ClassRule;
 import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
 
 /**
  * Impala shell connectivity tests with Simple Bind LDAP authentication.
+ *
+ * The test suite is parameterized, all tests are executed with both Kerberos
+ * authentication disabled and with Kerberos authentication enabled to validate
+ * that LDAP simple bind authentication is not broken even if Kerberos authentication
+ * is enabled.
  */
 @CreateDS(name = "myDS",
     partitions = { @CreatePartition(name = "test", suffix = "dc=myorg,dc=com") })
+@RunWith(Parameterized.class)
 public class LdapSimpleBindImpalaShellTest extends LdapImpalaShellTest {
+
+  @ClassRule
+  public static KerberosKdcEnvironment kerberosKdcEnvironment =
+          new KerberosKdcEnvironment(new TemporaryFolder());
+
+  private final boolean kerberosAuthenticationEnabled;
+
+  @Parameterized.Parameters(name = "kerberosAuthenticationEnabled={0}")
+  public static Boolean[] kerberosAuthenticationEnabled() {
+    return new Boolean[] {Boolean.FALSE, Boolean.TRUE};
+  }
+
+  public LdapSimpleBindImpalaShellTest(boolean isKerberosAuthenticationEnabled) {
+    this.kerberosAuthenticationEnabled = isKerberosAuthenticationEnabled;
+  }
+
+  @Override
+  protected int startImpalaCluster(String args) throws IOException, InterruptedException {
+    if (kerberosAuthenticationEnabled) {
+      return kerberosKdcEnvironment.startImpalaClusterWithArgs(args);
+    } else {
+      return super.startImpalaCluster(args);
+    }
+  }
+
   @Override
   public void setUp(String extraArgs) throws Exception {
     String dn = "cn=#UID,ou=Users,dc=myorg,dc=com";
-    String simpleBindArgs = String.format("--ldap_bind_pattern='%s' %s", dn, extraArgs);
+    String simpleBindArgs = String.format("--ldap_bind_pattern='%s' %s %s", dn,
+            getKerberosArgs(), extraArgs);
     super.setUp(simpleBindArgs);
   }
 
+  private String getKerberosArgs() throws IOException {
+    return kerberosAuthenticationEnabled ?
+            flagsToArgs(mergeFlags(
+                    kerberosKdcEnvironment.getKerberosAuthFlags(),
+                    ImmutableMap.of(
+                            "allow_custom_ldap_filters_with_kerberos_auth", "true"
+                    )
+            ))
+            :
+            "";  // empty if Kerberos authentication is disabled
+  }
+
   /**
    * Tests ldap authentication using impala-shell.
    */
diff --git a/fe/src/test/java/org/apache/impala/customcluster/RunShellCommand.java b/fe/src/test/java/org/apache/impala/customcluster/RunShellCommand.java
index 4d1291ce1..9dee84373 100644
--- a/fe/src/test/java/org/apache/impala/customcluster/RunShellCommand.java
+++ b/fe/src/test/java/org/apache/impala/customcluster/RunShellCommand.java
@@ -35,8 +35,20 @@ class RunShellCommand {
    */
   public static String Run(String[] cmd, boolean shouldSucceed, String expectedOut,
       String expectedErr) throws Exception {
+    // run the command with the env variables inherited from the current process
+    return Run(cmd, null, shouldSucceed, expectedOut, expectedErr);
+  }
+
+  /**
+   * Run a shell command 'cmd' with custom 'env' variables.
+   * If 'shouldSucceed' is true, the command is expected to
+   * succeed, otherwise it is expected to fail. Returns the output (stdout) of the
+   * command.
+   */
+  public static String Run(String[] cmd, String[] env, boolean shouldSucceed,
+                           String expectedOut, String expectedErr) throws Exception {
     Runtime rt = Runtime.getRuntime();
-    Process process = rt.exec(cmd);
+    Process process = rt.exec(cmd, env);
     // Collect stderr.
     BufferedReader input = new BufferedReader(
         new InputStreamReader(process.getErrorStream()));
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 625abb133..1df057957 100644
--- a/fe/src/test/java/org/apache/impala/testutil/LdapUtil.java
+++ b/fe/src/test/java/org/apache/impala/testutil/LdapUtil.java
@@ -38,6 +38,8 @@ public class LdapUtil {
   public static final String TEST_USER_6 = "Ldap\\, (Test6*)";
   public static final String TEST_PASSWORD_6 = "qrstu";
 
+  public static final String TEST_USER_7 = "Test7Ldap";
+
   // TEST_USER_1 and TEST_USER_2 are members of this group.
   public static final String TEST_USER_GROUP = "group1";
 
diff --git a/fe/src/test/resources/adschema.ldif b/fe/src/test/resources/adschema.ldif
new file mode 100644
index 000000000..855dff5c4
--- /dev/null
+++ b/fe/src/test/resources/adschema.ldif
@@ -0,0 +1,57 @@
+dn: cn=microsoft, ou=schema
+objectclass: metaSchema
+objectclass: top
+cn: microsoft
+
+dn: ou=attributetypes, cn=microsoft, ou=schema
+objectclass: organizationalUnit
+objectclass: top
+ou: attributetypes
+
+dn: m-oid=1.2.840.113556.1.4.221, ou=attributetypes, cn=microsoft, ou=schema
+objectclass: metaAttributeType
+objectclass: metaTop
+objectclass: top
+m-oid: 1.2.840.113556.1.4.221
+m-name: sAMAccountName
+m-equality: caseIgnoreMatch
+m-syntax: 1.3.6.1.4.1.1466.115.121.1.15
+m-singleValue: TRUE
+
+dn: m-oid=1.2.840.113556.1.4.222, ou=attributetypes, cn=microsoft, ou=schema
+objectclass: metaAttributeType
+objectclass: metaTop
+objectclass: top
+m-oid: 1.2.840.113556.1.4.222
+m-name: memberOf
+m-equality: caseIgnoreMatch
+m-syntax: 1.3.6.1.4.1.1466.115.121.1.15
+m-singleValue: FALSE
+
+dn: ou=objectClasses, cn=microsoft, ou=schema
+objectclass: organizationalUnit
+objectclass: top
+ou: objectClasses
+
+dn: m-oid=1.2.840.113556.1.5.9, ou=objectClasses, cn=microsoft, ou=schema
+objectclass: metaObjectClass
+objectclass: metaTop
+objectclass: top
+m-oid:1.2.840.113556.1.5.9
+m-name: user
+m-supObjectClass: top
+m-must: cn
+m-must: uid
+m-must: userPassword
+m-must: sAMAccountName
+m-may: memberOf
+
+dn: m-oid=1.2.840.113556.1.5.8, ou=objectClasses, cn=microsoft, ou=schema
+objectclass: metaObjectClass
+objectclass: metaTop
+objectclass: top
+m-oid:1.2.840.113556.1.5.8
+m-name: group
+m-supObjectClass: top
+m-must: cn
+m-may: member
diff --git a/fe/src/test/resources/adusers.ldif b/fe/src/test/resources/adusers.ldif
new file mode 100644
index 000000000..af79bf637
--- /dev/null
+++ b/fe/src/test/resources/adusers.ldif
@@ -0,0 +1,59 @@
+version: 1
+dn: dc=myorg,dc=com
+objectClass: domain
+objectClass: top
+dc: myorg
+
+dn: ou=Users,dc=myorg,dc=com
+objectClass: organizationalUnit
+objectClass: top
+ou: Users
+
+dn: ou=Users2,dc=myorg,dc=com
+objectClass: organizationalUnit
+objectClass: top
+ou: Users
+
+dn: ou=Groups,dc=myorg,dc=com
+objectClass: organizationalUnit
+objectClass: top
+ou: Groups
+
+dn: cn=Test1Ldap,ou=Users,dc=myorg,dc=com
+objectClass: user
+objectClass: top
+cn: Test1Ldap
+uid: ldaptest1
+sAMAccountName: Test1Ldap
+userPassword: 12345
+
+dn: cn=Test2Ldap,ou=Users2,dc=myorg,dc=com
+objectClass: user
+objectClass: top
+cn: Test2Ldap
+uid: ldaptest2
+sAMAccountName: Test2Ldap
+userPassword: abcde
+
+dn: cn=Test3Ldap,ou=Users,dc=myorg,dc=com
+objectClass: user
+objectClass: top
+cn: Test3Ldap
+uid: ldaptest3
+sAMAccountName: Test3Ldap
+userPassword: 67890
+
+dn: cn=Test4Ldap,ou=Users2,dc=myorg,dc=com
+objectClass: user
+objectClass: top
+cn: Test4Ldap
+uid: ldaptest4
+sAMAccountName: Test4Ldap
+userPassword: fghij
+
+dn: cn=group1,ou=Groups,dc=myorg,dc=com
+objectClass: top
+objectClass: group
+cn: group1
+member: cn=Test1Ldap,ou=Users,dc=myorg,dc=com
+member: cn=Test2Ldap,ou=Users2,dc=myorg,dc=com
diff --git a/fe/src/test/resources/users.ldif b/fe/src/test/resources/users.ldif
index d5e85beb6..c1787ef1b 100644
--- a/fe/src/test/resources/users.ldif
+++ b/fe/src/test/resources/users.ldif
@@ -79,12 +79,23 @@ sn: Ldap
 uid: ldaptes6
 userPassword: qrstu
 
+dn: cn=Test7Ldap,ou=Users,dc=myorg,dc=com
+objectClass: inetOrgPerson
+objectClass: organizationalPerson
+objectClass: person
+objectClass: top
+cn: Test7Ldap
+sn: Ldap
+uid: ldaptest7
+userPassword: vwxyz
+
 dn: cn=group1,ou=Groups,dc=myorg,dc=com
 objectClass: top
 objectClass: groupOfUniqueNames
 cn: group1
 uniqueMember: cn=Test1Ldap,ou=Users,dc=myorg,dc=com
 uniqueMember: cn=Test2Ldap,ou=Users,dc=myorg,dc=com
+uniqueMember: cn=Test7Ldap,ou=Users,dc=myorg,dc=com
 uniqueMember: cn=Ldap\, (Test6*),ou=Users2,dc=myorg,dc=com
 
 dn: cn=group2,ou=Groups,dc=myorg,dc=com


[impala] 02/02: IMPALA-11990: Make actual failures clearer

Posted by st...@apache.org.
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

commit 39fea06f2bca44cc2148e4fda3f72be2dfd6d280
Author: Michael Smith <mi...@cloudera.com>
AuthorDate: Mon Jan 30 10:44:30 2023 -0800

    IMPALA-11990: Make actual failures clearer
    
    A hack to cleanup after Hbase fails when services haven't been started
    yet (which is always at least once in a CI run) with a large error
    message. That error isn't useful and can be misleading for people
    reviewing test logs. Suppress it.
    
    Guards data load for Ozone as a usable snapshot is required. Also fixes
    a typo in fixed issues.
    
    Change-Id: Idc37d03780fca35427b977524b2b97a6892c88f7
    Reviewed-on: http://gerrit.cloudera.org:8080/19459
    Reviewed-by: Gabor Kaszab <ga...@cloudera.com>
    Tested-by: Michael Smith <mi...@cloudera.com>
---
 docs/topics/impala_fixed_issues.xml | 2 +-
 testdata/bin/create-load-data.sh    | 6 +++---
 testdata/bin/kill-hbase.sh          | 3 ++-
 3 files changed, 6 insertions(+), 5 deletions(-)

diff --git a/docs/topics/impala_fixed_issues.xml b/docs/topics/impala_fixed_issues.xml
index f9c88b569..542965d6d 100644
--- a/docs/topics/impala_fixed_issues.xml
+++ b/docs/topics/impala_fixed_issues.xml
@@ -150,7 +150,7 @@ under the License.
         <li><xref href="https://issues.apache.org/jira/browse/IMPALA-8061"
             format="html" scope="external">IMPALA-8061</xref> - Impala correctly
             initializes<codeph> S3_ACCESS_VALIDATED</codeph> variable to zero
-          when <codeph>TARGET_FILESYSTEM=3</codeph>.</li>
+          when <codeph>TARGET_FILESYSTEM=s3</codeph>.</li>
         <li><xref href="https://issues.apache.org/jira/browse/IMPALA-8154"
             format="html" scope="external">IMPALA-8154</xref> - Disabled the
             Kerberos<codeph> auth_to_local</codeph> setting to prevent
diff --git a/testdata/bin/create-load-data.sh b/testdata/bin/create-load-data.sh
index 3c8f6fc51..35b2e8322 100755
--- a/testdata/bin/create-load-data.sh
+++ b/testdata/bin/create-load-data.sh
@@ -136,10 +136,10 @@ elif [ $SKIP_SNAPSHOT_LOAD -eq 0 ]; then
     if [[ "${TARGET_FILESYSTEM}" == "isilon" || "${TARGET_FILESYSTEM}" == "s3" || \
           "${TARGET_FILESYSTEM}" == "local" || "${TARGET_FILESYSTEM}" == "gs" || \
           "${TARGET_FILESYSTEM}" == "cosn" || "${TARGET_FILESYSTEM}" == "oss" || \
-          "${TARGET_FILESYSTEM}" == "obs" ]] ; then
+          "${TARGET_FILESYSTEM}" == "obs" || "${TARGET_FILESYSTEM}" == "ozone" ]] ; then
       echo "ERROR in $0 at line $LINENO: A schema change has been detected in the"
-      echo "metadata, but it cannot be loaded on isilon, s3, gcs, cos, oss or local"
-      echo "and the target file system is ${TARGET_FILESYSTEM}.  Exiting."
+      echo "metadata, but it cannot be loaded on isilon, s3, gcs, cos, oss, obs, ozone,"
+      echo "or local and the target file system is ${TARGET_FILESYSTEM}.  Exiting."
       # Generate an explicit JUnitXML symptom report here for easier triaging
       ${IMPALA_HOME}/bin/generate_junitxml.py --phase=dataload \
           --step=check-schema-diff.sh --error "${SCHEMA_MISMATCH_ERROR}"
diff --git a/testdata/bin/kill-hbase.sh b/testdata/bin/kill-hbase.sh
index 2589d3032..c2fb3759d 100755
--- a/testdata/bin/kill-hbase.sh
+++ b/testdata/bin/kill-hbase.sh
@@ -38,4 +38,5 @@ rm -rf /tmp/hbase-*
 # HACK: Some jobs have seen the HBase master fail to initialize with mesages like:
 # "Master startup cannot progress, in holding-pattern until region onlined."
 # Anecdotally, removing the MasterProcWALs directory avoids the issue.
-hdfs dfs -Dozone.client.failover.max.attempts=3 -rm /hbase/MasterProcWALs/* || true
+hdfs dfs -Dozone.client.failover.max.attempts=3 -rm /hbase/MasterProcWALs/* \
+  >& /dev/null || true