You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kyuubi.apache.org by ch...@apache.org on 2024/01/19 17:12:58 UTC

(kyuubi) branch branch-1.8 updated: [KYUUBI #6001] Fix RESTful protocol security enabled evaluation

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

chengpan pushed a commit to branch branch-1.8
in repository https://gitbox.apache.org/repos/asf/kyuubi.git


The following commit(s) were added to refs/heads/branch-1.8 by this push:
     new 91f74538d [KYUUBI #6001] Fix RESTful protocol security enabled evaluation
91f74538d is described below

commit 91f74538da6a38c677bb61c9a4b7ab417624d456
Author: Cheng Pan <ch...@apache.org>
AuthorDate: Sat Jan 20 00:52:07 2024 +0800

    [KYUUBI #6001] Fix RESTful protocol security enabled evaluation
    
    https://github.com/apache/kyuubi/pull/5568#discussion_r1453616404
    
    Only when Kerberos is enabled or effectivePlainAuthType is not NONE, RESTful security is enabled
    
    - [x] Bugfix (non-breaking change which fixes an issue)
    - [ ] New feature (non-breaking change which adds functionality)
    - [ ] Breaking change (fix or feature that would cause existing functionality to change)
    
    when `kyuubi.authentication=KERBEROS` and use RESTful API, the exception is thrown `AuthenticationException("Kerberos is not supported for thrift http mode")`
    
    when `kyuubi.authentication=KERBEROS`, the RESTful API uses SPNego authN.
    
    Add some `*KyuubiRestFrontendServiceSuite`
    
    ---
    
    - [x] This patch was not authored or co-authored using [Generative Tooling](https://www.apache.org/legal/generative-tooling.html)
    
    **Be nice. Be informative.**
    
    Closes #6001 from pan3793/auth-krb.
    
    Closes #6001
    
    36404245f [Cheng Pan] nit
    69d33fb40 [Cheng Pan] fix
    f18cf8479 [Cheng Pan] Fix RESTful security enabled evaluation
    
    Authored-by: Cheng Pan <ch...@apache.org>
    Signed-off-by: Cheng Pan <ch...@apache.org>
---
 .../apache/kyuubi/service/TFrontendService.scala   |  6 +-
 .../kyuubi/service/authentication/AuthUtils.scala  | 72 ++++++++++++++++++++++
 .../KyuubiAuthenticationFactory.scala              | 60 ++----------------
 .../KyuubiAuthenticationFactorySuite.scala         |  5 +-
 .../kyuubi/server/KyuubiRestFrontendService.scala  | 13 ++--
 .../kyuubi/server/KyuubiTHttpFrontendService.scala | 10 +--
 .../server/KyuubiRestFrontendServiceSuite.scala    | 39 ++++++++++++
 .../server/api/v1/BatchesResourceSuite.scala       |  4 +-
 8 files changed, 131 insertions(+), 78 deletions(-)

diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/service/TFrontendService.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/service/TFrontendService.scala
index 7cc23779f..a375d6c6d 100644
--- a/kyuubi-common/src/main/scala/org/apache/kyuubi/service/TFrontendService.scala
+++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/service/TFrontendService.scala
@@ -34,7 +34,7 @@ import org.apache.kyuubi.Utils.stringifyException
 import org.apache.kyuubi.config.KyuubiConf.{FRONTEND_ADVERTISED_HOST, FRONTEND_CONNECTION_URL_USE_HOSTNAME, SESSION_CLOSE_ON_DISCONNECT}
 import org.apache.kyuubi.config.KyuubiReservedKeys._
 import org.apache.kyuubi.operation.{FetchOrientation, OperationHandle}
-import org.apache.kyuubi.service.authentication.KyuubiAuthenticationFactory
+import org.apache.kyuubi.service.authentication.{AuthUtils, KyuubiAuthenticationFactory}
 import org.apache.kyuubi.session.SessionHandle
 import org.apache.kyuubi.util.{KyuubiHadoopUtils, NamedThreadFactory}
 
@@ -127,11 +127,11 @@ abstract class TFrontendService(name: String)
       sessionConf: java.util.Map[String, String],
       ipAddress: String,
       realUser: String): String = {
-    val proxyUser = sessionConf.get(KyuubiAuthenticationFactory.HS2_PROXY_USER)
+    val proxyUser = sessionConf.get(AuthUtils.HS2_PROXY_USER)
     if (proxyUser == null) {
       realUser
     } else {
-      KyuubiAuthenticationFactory.verifyProxyAccess(realUser, proxyUser, ipAddress, hadoopConf)
+      AuthUtils.verifyProxyAccess(realUser, proxyUser, ipAddress, hadoopConf)
       proxyUser
     }
   }
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/AuthUtils.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/AuthUtils.scala
new file mode 100644
index 000000000..d3191ae23
--- /dev/null
+++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/AuthUtils.scala
@@ -0,0 +1,72 @@
+/*
+ * 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.kyuubi.service.authentication
+
+import java.io.IOException
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.security.authentication.util.KerberosName
+import org.apache.hadoop.security.authorize.ProxyUsers
+
+import org.apache.kyuubi.{KyuubiSQLException, Logging}
+import org.apache.kyuubi.service.authentication.AuthTypes.{AuthType, KERBEROS, NOSASL}
+
+object AuthUtils extends Logging {
+  val HS2_PROXY_USER = "hive.server2.proxy.user"
+
+  @throws[KyuubiSQLException]
+  def verifyProxyAccess(
+      realUser: String,
+      proxyUser: String,
+      ipAddress: String,
+      hadoopConf: Configuration): Unit = {
+    try {
+      val sessionUgi = {
+        if (UserGroupInformation.isSecurityEnabled) {
+          val kerbName = new KerberosName(realUser)
+          UserGroupInformation.createProxyUser(
+            kerbName.getServiceName,
+            UserGroupInformation.getLoginUser)
+        } else {
+          UserGroupInformation.createRemoteUser(realUser)
+        }
+      }
+
+      if (!proxyUser.equalsIgnoreCase(realUser)) {
+        ProxyUsers.refreshSuperUserGroupsConfiguration(hadoopConf)
+        ProxyUsers.authorize(UserGroupInformation.createProxyUser(proxyUser, sessionUgi), ipAddress)
+      }
+    } catch {
+      case e: IOException =>
+        throw KyuubiSQLException(
+          "Failed to validate proxy privilege of " + realUser + " for " + proxyUser,
+          e)
+    }
+  }
+
+  def saslDisabled(authTypes: Seq[AuthType]): Boolean = authTypes == Seq(NOSASL)
+
+  def kerberosEnabled(authTypes: Seq[AuthType]): Boolean = authTypes.contains(KERBEROS)
+
+  // take the first declared SASL/PLAIN auth type
+  def effectivePlainAuthType(authTypes: Seq[AuthType]): Option[AuthType] = authTypes.find {
+    case NOSASL | KERBEROS => false
+    case _ => true
+  }
+}
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/KyuubiAuthenticationFactory.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/KyuubiAuthenticationFactory.scala
index df6dc5d3e..eae89f48e 100644
--- a/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/KyuubiAuthenticationFactory.scala
+++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/KyuubiAuthenticationFactory.scala
@@ -21,31 +21,21 @@ import java.io.IOException
 import javax.security.auth.login.LoginException
 import javax.security.sasl.Sasl
 
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.security.UserGroupInformation
-import org.apache.hadoop.security.authentication.util.KerberosName
-import org.apache.hadoop.security.authorize.ProxyUsers
 import org.apache.hive.service.rpc.thrift.TCLIService.Iface
 import org.apache.thrift.TProcessorFactory
 import org.apache.thrift.transport.{TSaslServerTransport, TTransportException, TTransportFactory}
 
-import org.apache.kyuubi.{KyuubiSQLException, Logging}
+import org.apache.kyuubi.Logging
 import org.apache.kyuubi.config.KyuubiConf
 import org.apache.kyuubi.config.KyuubiConf._
-import org.apache.kyuubi.service.authentication.AuthMethods.AuthMethod
 import org.apache.kyuubi.service.authentication.AuthTypes._
 
 class KyuubiAuthenticationFactory(conf: KyuubiConf, isServer: Boolean = true) extends Logging {
 
   val authTypes: Seq[AuthType] = conf.get(AUTHENTICATION_METHOD).map(AuthTypes.withName)
-  val saslDisabled: Boolean = authTypes == Seq(NOSASL)
-  val kerberosEnabled: Boolean = authTypes.contains(KERBEROS)
-
-  // take the first declared SASL/PLAIN auth type
-  private val effectivePlainAuthType = authTypes.find {
-    case NOSASL | KERBEROS => false
-    case _ => true
-  }
+  val saslDisabled: Boolean = AuthUtils.saslDisabled(authTypes)
+  val kerberosEnabled: Boolean = AuthUtils.kerberosEnabled(authTypes)
+  val effectivePlainAuthType: Option[AuthType] = AuthUtils.effectivePlainAuthType(authTypes)
 
   private val hadoopAuthServer: Option[HadoopThriftAuthBridgeServer] = {
     if (kerberosEnabled) {
@@ -123,45 +113,3 @@ class KyuubiAuthenticationFactory(conf: KyuubiConf, isServer: Boolean = true) ex
       .orElse(Option(TSetIpAddressProcessor.getUserIpAddress))
   }
 }
-object KyuubiAuthenticationFactory extends Logging {
-  val HS2_PROXY_USER = "hive.server2.proxy.user"
-
-  @throws[KyuubiSQLException]
-  def verifyProxyAccess(
-      realUser: String,
-      proxyUser: String,
-      ipAddress: String,
-      hadoopConf: Configuration): Unit = {
-    try {
-      val sessionUgi = {
-        if (UserGroupInformation.isSecurityEnabled) {
-          val kerbName = new KerberosName(realUser)
-          UserGroupInformation.createProxyUser(
-            kerbName.getServiceName,
-            UserGroupInformation.getLoginUser)
-        } else {
-          UserGroupInformation.createRemoteUser(realUser)
-        }
-      }
-
-      if (!proxyUser.equalsIgnoreCase(realUser)) {
-        ProxyUsers.refreshSuperUserGroupsConfiguration(hadoopConf)
-        ProxyUsers.authorize(UserGroupInformation.createProxyUser(proxyUser, sessionUgi), ipAddress)
-      }
-    } catch {
-      case e: IOException =>
-        throw KyuubiSQLException(
-          "Failed to validate proxy privilege of " + realUser + " for " + proxyUser,
-          e)
-    }
-  }
-
-  def getValidPasswordAuthMethod(authTypes: Seq[AuthType]): AuthMethod = {
-    if (authTypes == Seq(NOSASL)) AuthMethods.NONE
-    else if (authTypes.contains(NONE)) AuthMethods.NONE
-    else if (authTypes.contains(LDAP)) AuthMethods.LDAP
-    else if (authTypes.contains(JDBC)) AuthMethods.JDBC
-    else if (authTypes.contains(CUSTOM)) AuthMethods.CUSTOM
-    else throw new IllegalArgumentException("No valid Password Auth detected")
-  }
-}
diff --git a/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/KyuubiAuthenticationFactorySuite.scala b/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/KyuubiAuthenticationFactorySuite.scala
index c63a1fb63..09eee4eb7 100644
--- a/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/KyuubiAuthenticationFactorySuite.scala
+++ b/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/KyuubiAuthenticationFactorySuite.scala
@@ -29,21 +29,20 @@ import org.apache.kyuubi.util.AssertionUtils._
 import org.apache.kyuubi.util.KyuubiHadoopUtils
 
 class KyuubiAuthenticationFactorySuite extends KyuubiFunSuite {
-  import KyuubiAuthenticationFactory._
 
   test("verify proxy access") {
     val kyuubiConf = KyuubiConf()
     val hadoopConf = KyuubiHadoopUtils.newHadoopConf(kyuubiConf)
 
     val e1 = intercept[KyuubiSQLException] {
-      verifyProxyAccess("kent", "yao", "localhost", hadoopConf)
+      AuthUtils.verifyProxyAccess("kent", "yao", "localhost", hadoopConf)
     }
     assert(e1.getMessage === "Failed to validate proxy privilege of kent for yao")
 
     kyuubiConf.set("hadoop.proxyuser.kent.groups", "*")
     kyuubiConf.set("hadoop.proxyuser.kent.hosts", "*")
     val hadoopConf2 = KyuubiHadoopUtils.newHadoopConf(kyuubiConf)
-    verifyProxyAccess("kent", "yao", "localhost", hadoopConf2)
+    AuthUtils.verifyProxyAccess("kent", "yao", "localhost", hadoopConf2)
   }
 
   test("AuthType NONE") {
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiRestFrontendService.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiRestFrontendService.scala
index 65b108468..cfbada792 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiRestFrontendService.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiRestFrontendService.scala
@@ -35,7 +35,7 @@ import org.apache.kyuubi.server.api.v1.ApiRootResource
 import org.apache.kyuubi.server.http.authentication.{AuthenticationFilter, KyuubiHttpAuthenticationFactory}
 import org.apache.kyuubi.server.ui.{JettyServer, JettyUtils}
 import org.apache.kyuubi.service.{AbstractFrontendService, Serverable, Service, ServiceUtils}
-import org.apache.kyuubi.service.authentication.{AuthMethods, AuthTypes, KyuubiAuthenticationFactory}
+import org.apache.kyuubi.service.authentication.{AuthTypes, AuthUtils}
 import org.apache.kyuubi.session.{KyuubiSessionManager, SessionHandle}
 import org.apache.kyuubi.util.ThreadUtils
 import org.apache.kyuubi.util.ThreadUtils.scheduleTolerableRunnableWithFixedDelay
@@ -71,9 +71,10 @@ class KyuubiRestFrontendService(override val serverable: Serverable)
 
   private lazy val port: Int = conf.get(FRONTEND_REST_BIND_PORT)
 
-  private lazy val securityEnabled = {
+  private[kyuubi] lazy val securityEnabled = {
     val authTypes = conf.get(AUTHENTICATION_METHOD).map(AuthTypes.withName)
-    KyuubiAuthenticationFactory.getValidPasswordAuthMethod(authTypes) != AuthMethods.NONE
+    AuthUtils.kerberosEnabled(authTypes) ||
+    !AuthUtils.effectivePlainAuthType(authTypes).contains(AuthTypes.NONE)
   }
 
   private lazy val administrators: Set[String] =
@@ -227,7 +228,7 @@ class KyuubiRestFrontendService(override val serverable: Serverable)
 
   def getSessionUser(hs2ProxyUser: String): String = {
     val sessionConf = Option(hs2ProxyUser).filter(_.nonEmpty).map(proxyUser =>
-      Map(KyuubiAuthenticationFactory.HS2_PROXY_USER -> proxyUser)).getOrElse(Map())
+      Map(AuthUtils.HS2_PROXY_USER -> proxyUser)).getOrElse(Map())
     getSessionUser(sessionConf)
   }
 
@@ -256,9 +257,9 @@ class KyuubiRestFrontendService(override val serverable: Serverable)
     if (sessionConf == null) {
       realUser
     } else {
-      sessionConf.get(KyuubiAuthenticationFactory.HS2_PROXY_USER).map { proxyUser =>
+      sessionConf.get(AuthUtils.HS2_PROXY_USER).map { proxyUser =>
         if (!isAdministrator(realUser)) {
-          KyuubiAuthenticationFactory.verifyProxyAccess(realUser, proxyUser, ipAddress, hadoopConf)
+          AuthUtils.verifyProxyAccess(realUser, proxyUser, ipAddress, hadoopConf)
         }
         proxyUser
       }.getOrElse(realUser)
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiTHttpFrontendService.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiTHttpFrontendService.scala
index c32345695..91bad6a85 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiTHttpFrontendService.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiTHttpFrontendService.scala
@@ -43,7 +43,6 @@ import org.apache.kyuubi.metrics.MetricsSystem
 import org.apache.kyuubi.server.http.ThriftHttpServlet
 import org.apache.kyuubi.server.http.util.SessionManager
 import org.apache.kyuubi.service.{Serverable, Service, ServiceUtils, TFrontendService}
-import org.apache.kyuubi.service.authentication.KyuubiAuthenticationFactory
 import org.apache.kyuubi.util.NamedThreadFactory
 
 /**
@@ -75,13 +74,8 @@ final class KyuubiTHttpFrontendService(
    */
   override def initialize(conf: KyuubiConf): Unit = synchronized {
     this.conf = conf
-    if (authFactory.kerberosEnabled) {
-      try {
-        KyuubiAuthenticationFactory.getValidPasswordAuthMethod(authFactory.authTypes)
-      } catch {
-        case _: IllegalArgumentException =>
-          throw new AuthenticationException("Kerberos is not supported for thrift http mode")
-      }
+    if (authFactory.kerberosEnabled && authFactory.effectivePlainAuthType.isEmpty) {
+      throw new AuthenticationException("Kerberos is not supported for Thrift HTTP mode")
     }
 
     try {
diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/KyuubiRestFrontendServiceSuite.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/KyuubiRestFrontendServiceSuite.scala
index 20dd863f9..b60517a06 100644
--- a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/KyuubiRestFrontendServiceSuite.scala
+++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/KyuubiRestFrontendServiceSuite.scala
@@ -19,9 +19,15 @@ package org.apache.kyuubi.server
 
 import org.apache.kyuubi.{KYUUBI_VERSION, RestFrontendTestHelper}
 import org.apache.kyuubi.client.api.v1.dto.VersionInfo
+import org.apache.kyuubi.config.KyuubiConf
+import org.apache.kyuubi.config.KyuubiConf._
+import org.apache.kyuubi.service.authentication.AnonymousAuthenticationProviderImpl
 
 class KyuubiRestFrontendServiceSuite extends RestFrontendTestHelper {
 
+  override protected lazy val conf: KyuubiConf = KyuubiConf()
+    .set(AUTHENTICATION_METHOD, Seq("NONE"))
+
   test("version") {
     val resp = v1Call("version")
     assert(resp.readEntity(classOf[VersionInfo]).getVersion === KYUUBI_VERSION)
@@ -51,3 +57,36 @@ class KyuubiRestFrontendServiceSuite extends RestFrontendTestHelper {
     assert(resp.getStatus === 200)
   }
 }
+
+class KerberosKyuubiRestFrontendServiceSuite extends RestFrontendTestHelper {
+
+  override protected lazy val conf: KyuubiConf = KyuubiConf()
+    .set(AUTHENTICATION_METHOD, Seq("KERBEROS"))
+    .set(AUTHENTICATION_CUSTOM_CLASS, classOf[AnonymousAuthenticationProviderImpl].getName)
+
+  test("security enabled - KERBEROS") {
+    assert(fe.asInstanceOf[KyuubiRestFrontendService].securityEnabled === true)
+  }
+}
+
+class NoneKyuubiRestFrontendServiceSuite extends RestFrontendTestHelper {
+
+  override protected lazy val conf: KyuubiConf = KyuubiConf()
+    .set(AUTHENTICATION_METHOD, Seq("NONE"))
+    .set(AUTHENTICATION_CUSTOM_CLASS, classOf[AnonymousAuthenticationProviderImpl].getName)
+
+  test("security enabled - NONE") {
+    assert(fe.asInstanceOf[KyuubiRestFrontendService].securityEnabled === false)
+  }
+}
+
+class KerberosAndCustomKyuubiRestFrontendServiceSuite extends RestFrontendTestHelper {
+
+  override protected lazy val conf: KyuubiConf = KyuubiConf()
+    .set(AUTHENTICATION_METHOD, Seq("KERBEROS,CUSTOM"))
+    .set(AUTHENTICATION_CUSTOM_CLASS, classOf[AnonymousAuthenticationProviderImpl].getName)
+
+  test("security enabled - KERBEROS,CUSTOM") {
+    assert(fe.asInstanceOf[KyuubiRestFrontendService].securityEnabled === true)
+  }
+}
diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/BatchesResourceSuite.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/BatchesResourceSuite.scala
index f483b18b1..f20c3edda 100644
--- a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/BatchesResourceSuite.scala
+++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/BatchesResourceSuite.scala
@@ -45,7 +45,7 @@ import org.apache.kyuubi.operation.OperationState.OperationState
 import org.apache.kyuubi.server.{KyuubiBatchService, KyuubiRestFrontendService}
 import org.apache.kyuubi.server.http.util.HttpAuthUtils.{basicAuthorizationHeader, AUTHORIZATION_HEADER}
 import org.apache.kyuubi.server.metadata.api.{Metadata, MetadataFilter}
-import org.apache.kyuubi.service.authentication.{AnonymousAuthenticationProviderImpl, KyuubiAuthenticationFactory}
+import org.apache.kyuubi.service.authentication.{AnonymousAuthenticationProviderImpl, AuthUtils}
 import org.apache.kyuubi.session.{KyuubiBatchSession, KyuubiSessionManager, SessionHandle, SessionType}
 
 class BatchesV1ResourceSuite extends BatchesResourceSuiteBase {
@@ -129,7 +129,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite
     assert(batch.getEndTime === 0)
 
     requestObj.setConf((requestObj.getConf.asScala ++
-      Map(KyuubiAuthenticationFactory.HS2_PROXY_USER -> "root")).asJava)
+      Map(AuthUtils.HS2_PROXY_USER -> "root")).asJava)
     val proxyUserRequest = requestObj
     val proxyUserResponse = webTarget.path("api/v1/batches")
       .request(MediaType.APPLICATION_JSON_TYPE)