From bd83f897bbbfa8bc0cbef8b17edb4b0032fd3997 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Sat, 20 Jan 2024 00:52:07 +0800 Subject: [PATCH] [KYUUBI #6001] Fix RESTful protocol security enabled evaluation MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit # :mag: Description ## Issue References ๐Ÿ”— https://github.com/apache/kyuubi/pull/5568#discussion_r1453616404 ## Describe Your Solution ๐Ÿ”ง Only when Kerberos is enabled or effectivePlainAuthType is not NONE, RESTful security is enabled ## Types of changes :bookmark: - [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) ## Test Plan ๐Ÿงช #### Behavior Without This Pull Request :coffin: when `kyuubi.authentication=KERBEROS` and use RESTful API, the exception is thrown `AuthenticationException("Kerberos is not supported for thrift http mode")` #### Behavior With This Pull Request :tada: when `kyuubi.authentication=KERBEROS`, the RESTful API uses SPNego authN. #### Related Unit Tests Add some `*KyuubiRestFrontendServiceSuite` --- # Checklist ๐Ÿ“ - [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 Signed-off-by: Cheng Pan --- .../kyuubi/service/TFrontendService.scala | 6 +- .../service/authentication/AuthUtils.scala | 72 +++++++++++++++++++ .../KyuubiAuthenticationFactory.scala | 61 ++-------------- .../KyuubiAuthenticationFactorySuite.scala | 5 +- .../server/KyuubiRestFrontendService.scala | 11 +-- .../server/KyuubiTHttpFrontendService.scala | 10 +-- .../KyuubiRestFrontendServiceSuite.scala | 39 ++++++++++ .../server/api/v1/BatchesResourceSuite.scala | 6 +- 8 files changed, 131 insertions(+), 79 deletions(-) create mode 100644 kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/AuthUtils.scala 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 a742993c5ad..9aefe63c8b6 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 @@ -30,7 +30,7 @@ import org.apache.kyuubi.Utils.stringifyException import org.apache.kyuubi.config.KyuubiConf.{FRONTEND_ADVERTISED_HOST, FRONTEND_CONNECTION_URL_USE_HOSTNAME, PROXY_USER, 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.shaded.hive.service.rpc.thrift._ import org.apache.kyuubi.shaded.thrift.protocol.TProtocol @@ -128,11 +128,11 @@ abstract class TFrontendService(name: String) ipAddress: String, realUser: String): String = { val proxyUser = Option(sessionConf.get(PROXY_USER.key)) - .getOrElse(sessionConf.get(KyuubiAuthenticationFactory.HS2_PROXY_USER)) + .getOrElse(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 00000000000..d3191ae23ef --- /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 db72e105961..978527b8818 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,15 +21,9 @@ 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.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._ import org.apache.kyuubi.shaded.hive.service.rpc.thrift.TCLIService.Iface import org.apache.kyuubi.shaded.thrift.TProcessorFactory @@ -38,14 +32,9 @@ import org.apache.kyuubi.shaded.thrift.transport.{TSaslServerTransport, TTranspo 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 +112,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 e9aad2b948f..18520b39b64 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 @@ -28,21 +28,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 d738995130b..83aee66fef0 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] = @@ -259,9 +260,9 @@ class KyuubiRestFrontendService(override val serverable: Serverable) } else { val proxyUser = sessionConf.getOrElse( PROXY_USER.key, - sessionConf.getOrElse(KyuubiAuthenticationFactory.HS2_PROXY_USER, realUser)) + sessionConf.getOrElse(AuthUtils.HS2_PROXY_USER, realUser)) if (!proxyUser.equals(realUser) && !isAdministrator(realUser)) { - KyuubiAuthenticationFactory.verifyProxyAccess(realUser, proxyUser, ipAddress, hadoopConf) + AuthUtils.verifyProxyAccess(realUser, proxyUser, ipAddress, hadoopConf) } proxyUser } 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 ca8939d69a3..2763e9481f9 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 @@ -41,7 +41,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.shaded.hive.service.rpc.thrift.{TCLIService, TOpenSessionReq} import org.apache.kyuubi.shaded.thrift.protocol.TBinaryProtocol 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 20dd863f9cd..b60517a06d8 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 6d33093997c..f1ee71bec9f 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} import org.apache.kyuubi.shaded.hive.service.rpc.thrift.TProtocolVersion @@ -130,7 +130,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) @@ -856,7 +856,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite conf += (PROXY_USER.key -> username) } hs2ProxyUser.map { username => - conf += (KyuubiAuthenticationFactory.HS2_PROXY_USER -> username) + conf += (AuthUtils.HS2_PROXY_USER -> username) } val proxyUserRequest = newSparkBatchRequest(conf.toMap)