You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by cs...@apache.org on 2023/09/15 11:30:02 UTC

[impala] branch master updated (4be517e15 -> 6f9a7622e)

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

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


    from 4be517e15 IMPALA-12441: Simplify local toolchain development
     new 49a0445b5 IMPALA-12383: (Addendum) Use named params
     new 6f9a7622e IMPALA-12403: Fix Kerberos authentication when connecting with a proxy user that does not delegate another user

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.cc                       | 28 ++++++++++------------
 be/src/rpc/thrift-server.h                         |  3 +++
 be/src/service/impala-hs2-server.cc                | 22 ++++++++++++++++-
 be/src/transport/TSasl.cpp                         | 13 ++++++++++
 be/src/transport/TSasl.h                           |  4 ++++
 be/src/transport/TSaslTransport.cpp                |  4 ++++
 be/src/transport/TSaslTransport.h                  |  7 ++++++
 be/src/util/auth-util.cc                           |  8 +++++++
 be/src/util/auth-util.h                            |  4 ++++
 .../impala/customcluster/LdapImpalaShellTest.java  | 20 +++++++++++-----
 .../customcluster/LdapKerberosImpalaShellTest.java | 12 ++++++++++
 .../LdapSearchBindImpalaShellTest.java             | 25 +++++++++++++++++++
 tests/query_test/test_aggregation.py               |  2 +-
 13 files changed, 129 insertions(+), 23 deletions(-)


[impala] 02/02: IMPALA-12403: Fix Kerberos authentication when connecting with a proxy user that does not delegate another user

Posted by cs...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 6f9a7622ed42538591c4268aa0444acbe972d304
Author: Gergely Farkas <gf...@cloudera.com>
AuthorDate: Fri Aug 25 10:10:14 2023 +0200

    IMPALA-12403: Fix Kerberos authentication when connecting with a proxy
    user that does not delegate another user
    
    This change improves LDAP checking in the Hiveserver2 API where
    previously LDAP filter checks were run on the user principal received
    in Kerberos authentication, but after the modification they are run on
    the short username taken from the Kerberos principal.
    
    Tested with new custom cluster tests.
    
    Change-Id: I0141cd341cda84ed40cd72f8a038c8d5a215e5e1
    Reviewed-on: http://gerrit.cloudera.org:8080/20421
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
    Reviewed-by: Norbert Luksa <no...@cloudera.com>
---
 be/src/rpc/authentication.cc                       | 28 ++++++++++------------
 be/src/rpc/thrift-server.h                         |  3 +++
 be/src/service/impala-hs2-server.cc                | 22 ++++++++++++++++-
 be/src/transport/TSasl.cpp                         | 13 ++++++++++
 be/src/transport/TSasl.h                           |  4 ++++
 be/src/transport/TSaslTransport.cpp                |  4 ++++
 be/src/transport/TSaslTransport.h                  |  7 ++++++
 be/src/util/auth-util.cc                           |  8 +++++++
 be/src/util/auth-util.h                            |  4 ++++
 .../impala/customcluster/LdapImpalaShellTest.java  | 20 +++++++++++-----
 .../customcluster/LdapKerberosImpalaShellTest.java | 12 ++++++++++
 .../LdapSearchBindImpalaShellTest.java             | 25 +++++++++++++++++++
 12 files changed, 128 insertions(+), 22 deletions(-)

diff --git a/be/src/rpc/authentication.cc b/be/src/rpc/authentication.cc
index cdc390916..7de625c8c 100644
--- a/be/src/rpc/authentication.cc
+++ b/be/src/rpc/authentication.cc
@@ -500,16 +500,6 @@ 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
@@ -715,11 +705,12 @@ bool JWTTokenAuth(ThriftServer::ConnectionContext* connection_context,
   return true;
 }
 
-// Performs a step of SPNEGO auth for the HTTP transport and sets the username on
-// 'connection_context' if auth is successful. 'header_token' is the value from an
-// 'Authorization: Negotiate" header. Returns true if the step was successful and sets
-// 'is_complete' to indicate if more steps are needed. Returns false if an error was
-// encountered and the connection should be closed.
+// Performs a step of SPNEGO auth for the HTTP transport and sets the username and
+// kerberos_user_principal on 'connection_context' if auth is successful.
+// 'header_token' is the value from an 'Authorization: Negotiate" header.
+// Returns true if the step was successful and sets 'is_complete' to indicate
+// if more steps are needed. Returns false if an error was encountered and the
+// connection should be closed.
 bool NegotiateAuth(ThriftServer::ConnectionContext* connection_context,
     const AuthenticationHash& hash, const std::string& header_token, bool* is_complete) {
   if (header_token.empty()) {
@@ -774,6 +765,8 @@ bool NegotiateAuth(ThriftServer::ConnectionContext* connection_context,
 
         // Authentication was successful, so set the username on the connection.
         connection_context->username = username;
+        // Save the username as Kerberos user principal in the connection context.
+        connection_context->kerberos_user_principal = username;
         // Create a cookie to return.
         connection_context->return_headers.push_back(
             Substitute("Set-Cookie: $0", GenerateCookie(username, hash)));
@@ -1367,6 +1360,11 @@ void SecureAuthProvider::SetupConnectionContext(
       socket = down_cast<TSocket*>(sasl_transport->getUnderlyingTransport().get());
       // Get the username from the transport.
       connection_ptr->username = sasl_transport->getUsername();
+      if (sasl_transport->getMechanismName() == KERBEROS_MECHANISM) {
+        // Save the username as Kerberos user principal in the connection context
+        // if the actual auth mechanism is Kerberos.
+        connection_ptr->kerberos_user_principal = connection_ptr->username;
+      }
       break;
     }
     case ThriftServer::HTTP: {
diff --git a/be/src/rpc/thrift-server.h b/be/src/rpc/thrift-server.h
index 8b70cd270..f7e6bf615 100644
--- a/be/src/rpc/thrift-server.h
+++ b/be/src/rpc/thrift-server.h
@@ -114,6 +114,9 @@ class ThriftServer {
     std::string saml_relay_state;
     std::unique_ptr<TWrappedHttpRequest> request;
     std::unique_ptr<TWrappedHttpResponse> response;
+    // Used in case of Kerberos authentication only to store the authenticated Kerberos
+    // user principal
+    std::string kerberos_user_principal;
   };
 
   /// Interface class for receiving connection creation / termination events.
diff --git a/be/src/service/impala-hs2-server.cc b/be/src/service/impala-hs2-server.cc
index 1000caec8..5b15c1e66 100644
--- a/be/src/service/impala-hs2-server.cc
+++ b/be/src/service/impala-hs2-server.cc
@@ -366,6 +366,8 @@ void ImpalaServer::OpenSession(TOpenSessionResp& return_val,
         // If the current user is a valid proxy user, he/she can optionally perform
         // authorization requests on behalf of another user. This is done by setting
         // the 'impala.doas.user' Hive Server 2 configuration property.
+        // Note: The 'impala.doas.user' configuration overrides the user specified
+        // in the 'doAs' request parameter, which can be specified for hs2-http transport.
         state->do_as_user = v.second;
         Status status = AuthorizeProxyUser(state->connected_user, state->do_as_user);
         HS2_RETURN_IF_ERROR(return_val, status, SQLSTATE_GENERAL_ERROR);
@@ -394,9 +396,27 @@ 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.
+  // When no 'doAs' user is specified and Kerberos authentication is enabled, then the
+  // connected user is a Kerberos user principal and that will certainly not pass the
+  // LDAP checks, so in this case the LDAP filters will be checked with the short username
+  // derived from the Kerberos user principal.
   if (FLAGS_enable_ldap_auth && IsAuthorizedProxyUser(state->connected_user)) {
+    string username_for_ldap_filters = GetEffectiveUser(*state);
+    if (state->do_as_user.empty() &&
+        !connection_context->kerberos_user_principal.empty()) {
+
+      string short_username = GetShortUsernameFromKerberosPrincipal(
+        connection_context->kerberos_user_principal);
+      VLOG(1) << "No doAs user is specified and Kerberos authentication is enabled, "
+              << "the authenticated user principal is \""
+              << connection_context->kerberos_user_principal << "\". "
+              << "LDAP filters will be checked using the short username \""
+              << short_username << "\", which is extracted from the user principal.";
+      username_for_ldap_filters = short_username;
+    }
+
     bool success =
-        AuthManager::GetInstance()->GetLdap()->LdapCheckFilters(GetEffectiveUser(*state));
+      AuthManager::GetInstance()->GetLdap()->LdapCheckFilters(username_for_ldap_filters);
     if (!success) {
       HS2_RETURN_ERROR(return_val, "User is not authorized.", SQLSTATE_GENERAL_ERROR);
     }
diff --git a/be/src/transport/TSasl.cpp b/be/src/transport/TSasl.cpp
index 6ce9313e3..67dde2f45 100644
--- a/be/src/transport/TSasl.cpp
+++ b/be/src/transport/TSasl.cpp
@@ -256,5 +256,18 @@ uint8_t* TSaslServer::evaluateChallengeOrResponse(const uint8_t* response,
   *resLen = outlen;
   return out;
 }
+
+string TSaslServer::getMechanismName() {
+  const char* mechName;
+  int result =
+      sasl_getprop(conn, SASL_MECHNAME, reinterpret_cast<const void **>(&mechName));
+  if (result != SASL_OK) {
+    stringstream ss;
+    ss << "Error getting SASL_MECHNAME property: " << sasl_errstring(result, NULL, NULL);
+    throw SaslException(ss.str().c_str());
+  }
+  string ret(mechName);
+  return ret;
+}
 };
 #endif
diff --git a/be/src/transport/TSasl.h b/be/src/transport/TSasl.h
index d4671bb81..52b477fe2 100644
--- a/be/src/transport/TSasl.h
+++ b/be/src/transport/TSasl.h
@@ -229,6 +229,10 @@ class TSaslServer : public sasl::TSasl {
   /* Evaluates the response data and generates a challenge. */
   virtual uint8_t* evaluateChallengeOrResponse(const uint8_t* challenge,
                                                const uint32_t len, uint32_t* resLen);
+
+  /* Returns the active IANA-registered mechanism name of this SASL server. */
+  virtual std::string getMechanismName();
+
  private:
   /* The domain of the user agent */
   std::string userRealm;
diff --git a/be/src/transport/TSaslTransport.cpp b/be/src/transport/TSaslTransport.cpp
index 7778a502b..6d79e16a4 100644
--- a/be/src/transport/TSaslTransport.cpp
+++ b/be/src/transport/TSaslTransport.cpp
@@ -72,6 +72,10 @@ namespace apache { namespace thrift { namespace transport {
     return sasl_->getUsername();
   }
 
+  string TSaslTransport::getMechanismName() {
+    return sasl_->getMechanismName();
+  }
+
   void TSaslTransport::doSaslNegotiation() {
     NegotiationStatus status = TSASL_INVALID;
     uint32_t resLength;
diff --git a/be/src/transport/TSaslTransport.h b/be/src/transport/TSaslTransport.h
index b80e6701f..c30be2a0e 100644
--- a/be/src/transport/TSaslTransport.h
+++ b/be/src/transport/TSaslTransport.h
@@ -144,6 +144,13 @@ class TSaslTransport : public TVirtualTransport<TSaslTransport> {
    */
   std::string getUsername();
 
+  /**
+   * Returns the IANA-registered mechanism name from underlying sasl connection.
+   *
+   * @throws TTransportException if an error occurs
+   */
+  std::string getMechanismName();
+
  protected:
   /// Underlying transport
   std::shared_ptr<TTransport> transport_;
diff --git a/be/src/util/auth-util.cc b/be/src/util/auth-util.cc
index c2a4e15a7..bb378ce62 100644
--- a/be/src/util/auth-util.cc
+++ b/be/src/util/auth-util.cc
@@ -97,4 +97,12 @@ Status ParseKerberosPrincipal(const string& principal, string* service_name,
   return Status::OK();
 }
 
+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;
+}
+
 }
diff --git a/be/src/util/auth-util.h b/be/src/util/auth-util.h
index 71fb3f3d5..2070d9989 100644
--- a/be/src/util/auth-util.h
+++ b/be/src/util/auth-util.h
@@ -67,6 +67,10 @@ Status GetExternalKerberosPrincipal(std::string* out_principal);
 Status ParseKerberosPrincipal(const std::string& principal, std::string* service_name,
     std::string* hostname, std::string* realm);
 
+// Takes a Kerberos principal (either user/hostname@realm or user@realm)
+// and returns the username part.
+string GetShortUsernameFromKerberosPrincipal(const string& principal);
+
 /// Returns true if kerberos is enabled.
 inline bool IsKerberosEnabled() {
   return !FLAGS_principal.empty();
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 3cc6c8e31..3f61abe8b 100644
--- a/fe/src/test/java/org/apache/impala/customcluster/LdapImpalaShellTest.java
+++ b/fe/src/test/java/org/apache/impala/customcluster/LdapImpalaShellTest.java
@@ -79,6 +79,19 @@ public class LdapImpalaShellTest {
     return Boolean.parseBoolean(RunShellCommand.Run(cmd, true, "", "").replace("\n", ""));
   }
 
+  /**
+   * Returns list of transport protocols: "beeswax", "hs2" is always available,
+   * "hs2-http" is not available on older version of python.
+   */
+  protected List<String> getProtocolsToTest() 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");
+    }
+    return protocolsToTest;
+  }
+
   private void verifyMetrics(Range<Long> expectedBasicSuccess,
       Range<Long> expectedBasicFailure, Range<Long> expectedCookieSuccess,
       Range<Long> expectedCookieFailure) throws Exception {
@@ -154,13 +167,8 @@ public class LdapImpalaShellTest {
     String[] commandWithoutAuth = {
         "impala-shell.sh", "", String.format("--query=%s", query)};
     String protocolTemplate = "--protocol=%s";
-    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");
-    }
 
-    for (String p : protocolsToTest) {
+    for (String p : getProtocolsToTest()) {
       String protocol = String.format(protocolTemplate, p);
       validCommand[1] = protocol;
       RunShellCommand.Run(validCommand, /*shouldSucceed*/ true, TEST_USER_1,
diff --git a/fe/src/test/java/org/apache/impala/customcluster/LdapKerberosImpalaShellTest.java b/fe/src/test/java/org/apache/impala/customcluster/LdapKerberosImpalaShellTest.java
index 9b90f84bb..4be56f987 100644
--- a/fe/src/test/java/org/apache/impala/customcluster/LdapKerberosImpalaShellTest.java
+++ b/fe/src/test/java/org/apache/impala/customcluster/LdapKerberosImpalaShellTest.java
@@ -206,6 +206,10 @@ public class LdapKerberosImpalaShellTest extends LdapKerberosImpalaShellTestBase
             "/?doAs=" + delegateUser_,
             /* shouldSucceed */ true,
             delegateUser_, "");
+
+    // Proxy-user without delegation with different transport protocols.
+    testShellKerberosAuthWithUser(kerberosKdcEnvironment, TEST_USER_1,
+            /* shouldSucceed */ true);
   }
 
   /**
@@ -274,6 +278,10 @@ public class LdapKerberosImpalaShellTest extends LdapKerberosImpalaShellTestBase
             "/?doAs=" + TEST_USER_7,
             /* shouldSucceed */ true,
             TEST_USER_7, "");
+
+    // Proxy-user without delegation with different transport protocols.
+    testShellKerberosAuthWithUser(kerberosKdcEnvironment, TEST_USER_1,
+            /* shouldSucceed */ true);
   }
 
   /**
@@ -421,6 +429,10 @@ public class LdapKerberosImpalaShellTest extends LdapKerberosImpalaShellTestBase
             "/?doAs=" + delegateUser_,
             /* shouldSucceed */ true,
             delegateUser_, "");
+
+    // Proxy-user without delegation with different transport protocols.
+    testShellKerberosAuthWithUser(kerberosKdcEnvironment, TEST_USER_1,
+            /* shouldSucceed */ true);
   }
 
   /**
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 4d76471e4..b182e2985 100644
--- a/fe/src/test/java/org/apache/impala/customcluster/LdapSearchBindImpalaShellTest.java
+++ b/fe/src/test/java/org/apache/impala/customcluster/LdapSearchBindImpalaShellTest.java
@@ -166,6 +166,31 @@ public class LdapSearchBindImpalaShellTest extends LdapImpalaShellTest {
     testLdapFiltersWithProxyImpl();
   }
 
+  /**
+   * Tests proxy-user authentication without impersonation over all available protocols.
+   */
+  @Test
+  public void testLdapFiltersWithProxyWithoutDoAsUser() throws Exception {
+    // These correspond to the values in fe/src/test/resources/users.ldif
+    // Sets up a cluster where TEST_USER_1 can act as a proxy for any other user
+    // and TEST_USER_1 passes the group filter, and user filter, too.
+    setUp(String.format("--ldap_user_search_basedn=dc=myorg,dc=com "
+            + "--ldap_group_search_basedn=ou=Groups,dc=myorg,dc=com "
+            + "--ldap_user_filter=(&(objectClass=person)(cn={0})(!(cn=Test2Ldap))) "
+            + "--ldap_group_filter=(&(cn=group1)(uniqueMember={0})) "
+            + "--authorized_proxy_user_config=%s=* ", TEST_USER_1));
+
+    String query = "select logged_in_user()";
+
+    for (String protocol : getProtocolsToTest()) {
+      // Run as the proxy without a delegate user,
+      // testcase should pass since the proxy user passes the filters.
+      String[] command =
+          buildCommand(query, protocol, TEST_USER_1, TEST_PASSWORD_1, "/cliservice");
+      RunShellCommand.Run(command, /* shouldSucceed */ true, TEST_USER_1, "");
+    }
+  }
+
   /**
    * Test LDAP Search on multiple OUs.
    */


[impala] 01/02: IMPALA-12383: (Addendum) Use named params

Posted by cs...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 49a0445b5efe15992161144c33805055199c9969
Author: Michael Smith <mi...@cloudera.com>
AuthorDate: Thu Sep 14 12:12:32 2023 -0700

    IMPALA-12383: (Addendum) Use named params
    
    Updates the call to add_test_dimensions to pass named parameters to
    avoid any issues that might come up if additional parameters are added.
    
    Change-Id: Ief492d054abbd00338e27ded9988141db750707c
    Reviewed-on: http://gerrit.cloudera.org:8080/20483
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Michael Smith <mi...@cloudera.com>
---
 tests/query_test/test_aggregation.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/tests/query_test/test_aggregation.py b/tests/query_test/test_aggregation.py
index 2e20abe7e..c7498ec06 100644
--- a/tests/query_test/test_aggregation.py
+++ b/tests/query_test/test_aggregation.py
@@ -92,7 +92,7 @@ class TestAggregation(ImpalaTestSuite):
 
   @classmethod
   def add_test_dimensions(cls):
-    super(TestAggregation, cls).add_test_dimensions(ALL_CLUSTER_SIZES)
+    super(TestAggregation, cls).add_test_dimensions(cluster_sizes=ALL_CLUSTER_SIZES)
 
     # Add two more dimensions
     cls.ImpalaTestMatrix.add_dimension(ImpalaTestDimension('agg_func', *AGG_FUNCTIONS))