Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[KYUUBI #6041] RESTful API supports isolated authentication configuration #6042

Open
wants to merge 10 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions docs/configuration/settings.md
Original file line number Diff line number Diff line change
Expand Up @@ -246,6 +246,7 @@ You can configure the Kyuubi properties in `$KYUUBI_HOME/conf/kyuubi-defaults.co
| kyuubi.frontend.mysql.worker.keepalive.time | PT1M | Time(ms) that an idle async thread of the command execution thread pool will wait for a new task to arrive before terminating in MySQL frontend service | duration | 1.4.0 |
| kyuubi.frontend.protocols | THRIFT_BINARY,REST | A comma-separated list for all frontend protocols <ul> <li>THRIFT_BINARY - HiveServer2 compatible thrift binary protocol.</li> <li>THRIFT_HTTP - HiveServer2 compatible thrift http protocol.</li> <li>REST - Kyuubi defined REST API(experimental).</li> <li>MYSQL - MySQL compatible text protocol(experimental).</li> <li>TRINO - Trino compatible http protocol(experimental).</li> </ul> | seq | 1.4.0 |
| kyuubi.frontend.proxy.http.client.ip.header | X-Real-IP | The HTTP header to record the real client IP address. If your server is behind a load balancer or other proxy, the server will see this load balancer or proxy IP address as the client IP address, to get around this common issue, most load balancers or proxies offer the ability to record the real remote IP address in an HTTP header that will be added to the request for other devices to use. Note that, because the header value can be specified to any IP address, so it will not be used for authentication. | string | 1.6.0 |
| kyuubi.frontend.rest.authentication | NONE | A comma-separated list of client authentication types. It fallback to `kyuubi.authentication` if not configure. | seq | 1.9.0 |
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

mysql and other forntend protocols would implement in another PR.

| kyuubi.frontend.rest.bind.host | &lt;undefined&gt; | Hostname or IP of the machine on which to run the REST frontend service. | string | 1.4.0 |
| kyuubi.frontend.rest.bind.port | 10099 | Port of the machine on which to run the REST frontend service. | int | 1.4.0 |
| kyuubi.frontend.rest.max.worker.threads | 999 | Maximum number of threads in the frontend worker thread pool for the rest frontend service | int | 1.6.2 |
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -797,6 +797,14 @@ object KyuubiConf {
.checkValues(AuthTypes)
.createWithDefault(Seq(AuthTypes.NONE.toString))

val FRONTEND_REST_AUTHENTICATION_METHOD: ConfigEntry[Seq[String]] =
buildConf("kyuubi.frontend.rest.authentication")
.doc("A comma-separated list of client authentication types." +
" It fallback to `kyuubi.authentication` if not configure.")
.version("1.9.0")
.serverOnly
.fallbackConf(AUTHENTICATION_METHOD)

val AUTHENTICATION_CUSTOM_CLASS: OptionalConfigEntry[String] =
buildConf("kyuubi.authentication.custom.class")
Copy link
Member

@pan3793 pan3793 Feb 20, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should we make it isolated too?

Copy link
Contributor Author

@beryllw beryllw Feb 21, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

agree.

.doc("User-defined authentication implementation of " +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ class KyuubiRestFrontendService(override val serverable: Serverable)
private lazy val port: Int = conf.get(FRONTEND_REST_BIND_PORT)

private[kyuubi] lazy val securityEnabled = {
val authTypes = conf.get(AUTHENTICATION_METHOD).map(AuthTypes.withName)
val authTypes = conf.get(FRONTEND_REST_AUTHENTICATION_METHOD).map(AuthTypes.withName)
AuthUtils.kerberosEnabled(authTypes) ||
!AuthUtils.effectivePlainAuthType(authTypes).contains(AuthTypes.NONE)
}
Expand Down Expand Up @@ -103,7 +103,9 @@ class KyuubiRestFrontendService(override val serverable: Serverable)

private def startInternal(): Unit = {
val contextHandler = ApiRootResource.getServletHandler(this)
val holder = new FilterHolder(new AuthenticationFilter(conf))
val holder = new FilterHolder(new AuthenticationFilter(
conf,
conf.get(FRONTEND_REST_AUTHENTICATION_METHOD).map(AuthTypes.withName)))
contextHandler.addFilter(holder, "/v1/*", EnumSet.allOf(classOf[DispatcherType]))
val authenticationFactory = new KyuubiHttpAuthenticationFactory(conf)
server.addHandler(authenticationFactory.httpHandlerWrapperFactory.wrapHandler(contextHandler))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,11 +30,11 @@ import org.apache.hadoop.hive.shims.Utils

import org.apache.kyuubi.Logging
import org.apache.kyuubi.config.KyuubiConf
import org.apache.kyuubi.config.KyuubiConf.FRONTEND_PROXY_HTTP_CLIENT_IP_HEADER
import org.apache.kyuubi.config.KyuubiConf.{AUTHENTICATION_METHOD, FRONTEND_PROXY_HTTP_CLIENT_IP_HEADER}
import org.apache.kyuubi.server.http.authentication.AuthenticationFilter
import org.apache.kyuubi.server.http.util.{CookieSigner, HttpAuthUtils, SessionManager}
import org.apache.kyuubi.server.http.util.HttpAuthUtils.AUTHORIZATION_HEADER
import org.apache.kyuubi.service.authentication.KyuubiAuthenticationFactory
import org.apache.kyuubi.service.authentication.{AuthTypes, KyuubiAuthenticationFactory}
import org.apache.kyuubi.shaded.thrift.TProcessor
import org.apache.kyuubi.shaded.thrift.protocol.TProtocolFactory
import org.apache.kyuubi.shaded.thrift.server.TServlet
Expand All @@ -56,7 +56,8 @@ class ThriftHttpServlet(
private var isCookieSecure = false
private var isHttpOnlyCookie = false
private val X_FORWARDED_FOR_HEADER = "X-Forwarded-For"
private val authenticationFilter = new AuthenticationFilter(conf)
private val authenticationFilter =
new AuthenticationFilter(conf, conf.get(AUTHENTICATION_METHOD).map(AuthTypes.withName))

override def init(): Unit = {
isCookieAuthEnabled = conf.get(KyuubiConf.FRONTEND_THRIFT_HTTP_COOKIE_AUTH_ENABLED)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,12 +26,13 @@ import scala.collection.mutable

import org.apache.kyuubi.Logging
import org.apache.kyuubi.config.KyuubiConf
import org.apache.kyuubi.config.KyuubiConf.{AUTHENTICATION_METHOD, FRONTEND_PROXY_HTTP_CLIENT_IP_HEADER}
import org.apache.kyuubi.config.KyuubiConf.FRONTEND_PROXY_HTTP_CLIENT_IP_HEADER
import org.apache.kyuubi.server.http.util.HttpAuthUtils.AUTHORIZATION_HEADER
import org.apache.kyuubi.service.authentication.{AuthTypes, InternalSecurityAccessor}
import org.apache.kyuubi.service.authentication.AuthTypes.{KERBEROS, NOSASL}

class AuthenticationFilter(conf: KyuubiConf) extends Filter with Logging {
class AuthenticationFilter(conf: KyuubiConf, authTypes: Seq[AuthTypes.Value]) extends Filter
with Logging {
import AuthenticationFilter._
import AuthSchemes._

Expand All @@ -55,7 +56,6 @@ class AuthenticationFilter(conf: KyuubiConf) extends Filter with Logging {
}

private[kyuubi] def initAuthHandlers(): Unit = {
val authTypes = conf.get(AUTHENTICATION_METHOD).map(AuthTypes.withName)
val spnegoKerberosEnabled = authTypes.contains(KERBEROS)
val basicAuthTypeOpt = {
if (authTypes == Set(NOSASL)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,14 +25,14 @@ import org.eclipse.jetty.server.{Handler, Request}
import org.eclipse.jetty.server.handler.HandlerWrapper

import org.apache.kyuubi.config.KyuubiConf
import org.apache.kyuubi.config.KyuubiConf.{AUTHENTICATION_METHOD, ENGINE_SECURITY_ENABLED}
import org.apache.kyuubi.config.KyuubiConf.{ENGINE_SECURITY_ENABLED, FRONTEND_REST_AUTHENTICATION_METHOD}
import org.apache.kyuubi.metrics.MetricsConstants.{REST_CONN_FAIL, REST_CONN_OPEN, REST_CONN_TOTAL}
import org.apache.kyuubi.metrics.MetricsSystem
import org.apache.kyuubi.service.authentication.{AuthTypes, InternalSecurityAccessor}
import org.apache.kyuubi.service.authentication.AuthTypes.KERBEROS

class KyuubiHttpAuthenticationFactory(conf: KyuubiConf) {
private val authTypes = conf.get(AUTHENTICATION_METHOD).map(AuthTypes.withName)
private val authTypes = conf.get(FRONTEND_REST_AUTHENTICATION_METHOD).map(AuthTypes.withName)
private val kerberosEnabled = authTypes.contains(KERBEROS)
private val ugi = UserGroupInformation.getCurrentUser

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,9 @@ trait RestClientTestHelper extends RestFrontendTestHelper with KerberizedTestHel
UserGroupInformation.setConfiguration(config)
assert(UserGroupInformation.isSecurityEnabled)

val conf = KyuubiConf().set(KyuubiConf.AUTHENTICATION_METHOD, Seq("KERBEROS", "LDAP", "CUSTOM"))
val conf = KyuubiConf().set(
KyuubiConf.FRONTEND_REST_AUTHENTICATION_METHOD,
Seq("KERBEROS", "LDAP", "CUSTOM"))
.set(KyuubiConf.SERVER_KEYTAB.key, testKeytab)
.set(KyuubiConf.SERVER_PRINCIPAL, testPrincipal)
.set(KyuubiConf.SERVER_SPNEGO_KEYTAB, testKeytab)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -182,3 +182,33 @@ class KyuubiRestAuthenticationSuite extends RestClientTestHelper {
assert(HttpServletResponse.SC_UNAUTHORIZED == response.getStatus)
}
}

class NoneKyuubiRestAuthenticationSuite extends RestClientTestHelper {

override protected val otherConfigs: Map[String, String] = {
Map(KyuubiConf.FRONTEND_REST_AUTHENTICATION_METHOD.key -> "NONE")
}

test("test disable restful api authentication") {
val response = webTarget.path("api/v1/sessions/count")
.request()
.get()

assert(HttpServletResponse.SC_OK == response.getStatus)
}
}

class KerberosKyuubiRestAuthenticationSuite extends RestClientTestHelper {

override protected val otherConfigs: Map[String, String] = {
Map(KyuubiConf.FRONTEND_REST_AUTHENTICATION_METHOD.key -> "KERBEROS")
}

test("test without authorization when rest api authentication with KERBEROS") {
val response = webTarget.path("api/v1/sessions/count")
.request()
.get()

assert(HttpServletResponse.SC_UNAUTHORIZED == response.getStatus)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ import org.apache.kyuubi.service.authentication.AnonymousAuthenticationProviderI
class KyuubiRestFrontendServiceSuite extends RestFrontendTestHelper {

override protected lazy val conf: KyuubiConf = KyuubiConf()
.set(AUTHENTICATION_METHOD, Seq("NONE"))
.set(FRONTEND_REST_AUTHENTICATION_METHOD, Seq("NONE"))

test("version") {
val resp = v1Call("version")
Expand Down Expand Up @@ -61,7 +61,7 @@ class KyuubiRestFrontendServiceSuite extends RestFrontendTestHelper {
class KerberosKyuubiRestFrontendServiceSuite extends RestFrontendTestHelper {

override protected lazy val conf: KyuubiConf = KyuubiConf()
.set(AUTHENTICATION_METHOD, Seq("KERBEROS"))
.set(FRONTEND_REST_AUTHENTICATION_METHOD, Seq("KERBEROS"))
.set(AUTHENTICATION_CUSTOM_CLASS, classOf[AnonymousAuthenticationProviderImpl].getName)

test("security enabled - KERBEROS") {
Expand All @@ -72,7 +72,7 @@ class KerberosKyuubiRestFrontendServiceSuite extends RestFrontendTestHelper {
class NoneKyuubiRestFrontendServiceSuite extends RestFrontendTestHelper {

override protected lazy val conf: KyuubiConf = KyuubiConf()
.set(AUTHENTICATION_METHOD, Seq("NONE"))
.set(FRONTEND_REST_AUTHENTICATION_METHOD, Seq("NONE"))
.set(AUTHENTICATION_CUSTOM_CLASS, classOf[AnonymousAuthenticationProviderImpl].getName)

test("security enabled - NONE") {
Expand All @@ -83,7 +83,7 @@ class NoneKyuubiRestFrontendServiceSuite extends RestFrontendTestHelper {
class KerberosAndCustomKyuubiRestFrontendServiceSuite extends RestFrontendTestHelper {

override protected lazy val conf: KyuubiConf = KyuubiConf()
.set(AUTHENTICATION_METHOD, Seq("KERBEROS,CUSTOM"))
.set(FRONTEND_REST_AUTHENTICATION_METHOD, Seq("KERBEROS,CUSTOM"))
.set(AUTHENTICATION_CUSTOM_CLASS, classOf[AnonymousAuthenticationProviderImpl].getName)

test("security enabled - KERBEROS,CUSTOM") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper {
private val engineMgr = new KyuubiApplicationManager()

override protected lazy val conf: KyuubiConf = KyuubiConf()
.set(AUTHENTICATION_METHOD, Seq("CUSTOM"))
.set(FRONTEND_REST_AUTHENTICATION_METHOD, Seq("CUSTOM"))
.set(AUTHENTICATION_CUSTOM_CLASS, classOf[AnonymousAuthenticationProviderImpl].getName)
.set(SERVER_ADMINISTRATORS, Set("admin001"))
.set(ENGINE_IDLE_TIMEOUT, Duration.ofMinutes(3).toMillis)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -85,7 +85,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite
override protected lazy val conf: KyuubiConf = {
val testResourceDir = Paths.get(sparkBatchTestResource.get).getParent
val kyuubiConf = KyuubiConf()
.set(AUTHENTICATION_METHOD, Seq("CUSTOM"))
.set(FRONTEND_REST_AUTHENTICATION_METHOD, Seq("CUSTOM"))
.set(AUTHENTICATION_CUSTOM_CLASS, classOf[AnonymousAuthenticationProviderImpl].getName)
.set(SERVER_ADMINISTRATORS, Set("admin"))
.set(BATCH_IMPL_VERSION, batchVersion)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,14 @@ package org.apache.kyuubi.server.http.authentication

import org.apache.kyuubi.KyuubiFunSuite
import org.apache.kyuubi.config.KyuubiConf
import org.apache.kyuubi.config.KyuubiConf.AUTHENTICATION_METHOD
import org.apache.kyuubi.service.authentication.AuthTypes

class AuthenticationFilterSuite extends KyuubiFunSuite {
test("add auth handler and destroy") {
val filter = new AuthenticationFilter(KyuubiConf())
val conf = KyuubiConf()
val filter =
new AuthenticationFilter(conf, conf.get(AUTHENTICATION_METHOD).map(AuthTypes.withName))
filter.addAuthHandler(new BasicAuthenticationHandler(null))
assert(filter.authSchemeHandlers.isEmpty)
filter.addAuthHandler(new BasicAuthenticationHandler(AuthTypes.LDAP))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ class AdminCtlSuite extends RestClientTestHelper with TestPrematureExit {
val id = UUID.randomUUID().toString
conf.set(HighAvailabilityConf.HA_NAMESPACE, "kyuubi_test")
conf.set(KyuubiConf.ENGINE_IDLE_TIMEOUT, 180000L)
conf.set(KyuubiConf.AUTHENTICATION_METHOD, Seq("LDAP", "CUSTOM"))
conf.set(KyuubiConf.FRONTEND_REST_AUTHENTICATION_METHOD, Seq("LDAP", "CUSTOM"))
conf.set(KyuubiConf.GROUP_PROVIDER, "hadoop")

val user = ldapUser
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ class AdminRestApiSuite extends RestClientTestHelper {
val id = UUID.randomUUID().toString
conf.set(HighAvailabilityConf.HA_NAMESPACE, "kyuubi_test")
conf.set(KyuubiConf.ENGINE_IDLE_TIMEOUT, 180000L)
conf.set(KyuubiConf.AUTHENTICATION_METHOD, Seq("LDAP", "CUSTOM"))
conf.set(KyuubiConf.FRONTEND_REST_AUTHENTICATION_METHOD, Seq("LDAP", "CUSTOM"))
conf.set(KyuubiConf.GROUP_PROVIDER, "hadoop")
val user = ldapUser
val engine =
Expand Down
Loading