You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kyuubi.apache.org by fe...@apache.org on 2022/04/11 15:35:48 UTC
[incubator-kyuubi] branch master updated: [KYUUBI #2300] Add http UGIAssuming handler wrapper for kerberos enabled restful frontend service
This is an automated email from the ASF dual-hosted git repository.
feiwang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-kyuubi.git
The following commit(s) were added to refs/heads/master by this push:
new 9bd910547 [KYUUBI #2300] Add http UGIAssuming handler wrapper for kerberos enabled restful frontend service
9bd910547 is described below
commit 9bd910547ceb1f4b80939f488af275a98b0b6502
Author: Fei Wang <fw...@ebay.com>
AuthorDate: Mon Apr 11 23:35:41 2022 +0800
[KYUUBI #2300] Add http UGIAssuming handler wrapper for kerberos enabled restful frontend service
### _Why are the changes needed?_
Wrap the http handler with PrivilegedAction for kerberos enabled use case.
### _How was this patch tested?_
- [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible
- [ ] Add screenshots for manual tests if appropriate
- [ ] [Run test](https://kyuubi.apache.org/docs/latest/develop_tools/testing.html#running-tests) locally before make a pull request
Closes #2300 from turboFei/ugi_wrapper.
Closes #2300
da074065 [Fei Wang] add ut
a56fe67b [Fei Wang] verify proxy user for rest open session
9e0f228f [Fei Wang] remove blank
8070d13c [Fei Wang] revert KyuubiAuthenticationFactory change
d40fa134 [Fei Wang] Add ugi assuming http handler wrapper for kerberos enabeld restful front
Authored-by: Fei Wang <fw...@ebay.com>
Signed-off-by: Fei Wang <fw...@ebay.com>
---
.../kyuubi/server/KyuubiRestFrontendService.scala | 34 ++++++++--
.../kyuubi/server/api/v1/SessionsResource.scala | 9 ++-
.../KyuubiHttpAuthenticationFactory.scala | 77 ++++++++++++++++++++++
.../org/apache/kyuubi/server/ui/JettyServer.scala | 28 ++------
.../operation/KyuubiRestAuthenticationSuite.scala | 23 ++++++-
5 files changed, 139 insertions(+), 32 deletions(-)
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 3cb1794d5..8fc79ad2c 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
@@ -21,15 +21,18 @@ import java.util.EnumSet
import java.util.concurrent.atomic.AtomicBoolean
import javax.servlet.DispatcherType
+import org.apache.hadoop.conf.Configuration
import org.eclipse.jetty.servlet.FilterHolder
import org.apache.kyuubi.{KyuubiException, Utils}
import org.apache.kyuubi.config.KyuubiConf
import org.apache.kyuubi.config.KyuubiConf.{FRONTEND_REST_BIND_HOST, FRONTEND_REST_BIND_PORT}
-import org.apache.kyuubi.server.api.v1.ApiRootResource
-import org.apache.kyuubi.server.http.authentication.AuthenticationFilter
+import org.apache.kyuubi.server.api.v1.{ApiRootResource, SessionOpenRequest}
+import org.apache.kyuubi.server.http.authentication.{AuthenticationFilter, KyuubiHttpAuthenticationFactory}
import org.apache.kyuubi.server.ui.JettyServer
-import org.apache.kyuubi.service.{AbstractFrontendService, Serverable, Service}
+import org.apache.kyuubi.service.{AbstractFrontendService, Serverable, Service, ServiceUtils}
+import org.apache.kyuubi.service.authentication.KyuubiAuthenticationFactory
+import org.apache.kyuubi.util.KyuubiHadoopUtils
/**
* A frontend service based on RESTful api via HTTP protocol.
@@ -42,6 +45,8 @@ class KyuubiRestFrontendService(override val serverable: Serverable)
private val isStarted = new AtomicBoolean(false)
+ private lazy val hadoopConf: Configuration = KyuubiHadoopUtils.newHadoopConf(conf)
+
override def initialize(conf: KyuubiConf): Unit = synchronized {
val host = conf.get(FRONTEND_REST_BIND_HOST)
.getOrElse(Utils.findLocalInetAddress.getHostAddress)
@@ -58,7 +63,8 @@ class KyuubiRestFrontendService(override val serverable: Serverable)
val contextHandler = ApiRootResource.getServletHandler(this)
val holder = new FilterHolder(new AuthenticationFilter(conf))
contextHandler.addFilter(holder, "/*", EnumSet.allOf(classOf[DispatcherType]))
- server.addHandler(contextHandler)
+ val authenticationFactory = new KyuubiHttpAuthenticationFactory(conf)
+ server.addHandler(authenticationFactory.httpHandlerWrapperFactory.wrapHandler(contextHandler))
server.addStaticHandler("org/apache/kyuubi/ui/static", "/static")
server.addRedirectHandler("/", "/static")
@@ -87,5 +93,25 @@ class KyuubiRestFrontendService(override val serverable: Serverable)
super.stop()
}
+ def getUserName(req: SessionOpenRequest): String = {
+ val realUser: String =
+ ServiceUtils.getShortName(Option(AuthenticationFilter.getUserName).getOrElse(req.user))
+ if (req.configs == null) {
+ realUser
+ } else {
+ getProxyUser(req.configs, Option(AuthenticationFilter.getUserIpAddress).orNull, realUser)
+ }
+ }
+
+ private def getProxyUser(
+ sessionConf: Map[String, String],
+ ipAddress: String,
+ realUser: String): String = {
+ sessionConf.get(KyuubiAuthenticationFactory.HS2_PROXY_USER).map { proxyUser =>
+ KyuubiAuthenticationFactory.verifyProxyAccess(realUser, proxyUser, ipAddress, hadoopConf)
+ proxyUser
+ }.getOrElse(realUser)
+ }
+
override val discoveryService: Option[Service] = None
}
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/SessionsResource.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/SessionsResource.scala
index 5a37f6f46..a0193fd85 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/SessionsResource.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/SessionsResource.scala
@@ -31,6 +31,7 @@ import org.apache.kyuubi.Logging
import org.apache.kyuubi.events.KyuubiEvent
import org.apache.kyuubi.operation.OperationHandle
import org.apache.kyuubi.server.api.ApiRequestContext
+import org.apache.kyuubi.server.http.authentication.AuthenticationFilter
import org.apache.kyuubi.session.SessionHandle
import org.apache.kyuubi.session.SessionHandle.parseSessionHandle
@@ -130,12 +131,14 @@ private[v1] class SessionsResource extends ApiRequestContext with Logging {
@POST
@Consumes(Array(MediaType.APPLICATION_JSON))
def openSession(request: SessionOpenRequest): SessionHandle = {
+ val userName = fe.getUserName(request)
+ val ipAddress = AuthenticationFilter.getUserIpAddress
fe.be.openSession(
TProtocolVersion.findByValue(request.protocolVersion),
- request.user,
+ userName,
request.password,
- request.ipAddr,
- request.configs)
+ ipAddress,
+ Option(request.configs).getOrElse(Map.empty[String, String]))
}
@ApiResponse(
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/authentication/KyuubiHttpAuthenticationFactory.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/authentication/KyuubiHttpAuthenticationFactory.scala
new file mode 100644
index 000000000..83023ddc5
--- /dev/null
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/authentication/KyuubiHttpAuthenticationFactory.scala
@@ -0,0 +1,77 @@
+/*
+ * 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.server.http.authentication
+
+import java.security.PrivilegedAction
+import javax.servlet.http.{HttpServletRequest, HttpServletResponse}
+
+import org.apache.hadoop.security.UserGroupInformation
+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.service.authentication.{AuthTypes, EngineSecurityAccessor}
+import org.apache.kyuubi.service.authentication.AuthTypes.KERBEROS
+
+class KyuubiHttpAuthenticationFactory(conf: KyuubiConf) {
+ private val authTypes = conf.get(AUTHENTICATION_METHOD).map(AuthTypes.withName)
+ private val kerberosEnabled = authTypes.contains(KERBEROS)
+ private val ugi = UserGroupInformation.getCurrentUser
+
+ if (conf.get(ENGINE_SECURITY_ENABLED)) {
+ EngineSecurityAccessor.initialize(conf, true)
+ }
+
+ private[kyuubi] val httpHandlerWrapperFactory =
+ new HttpHandlerWrapperFactory(ugi, kerberosEnabled)
+
+ class HttpHandlerWrapperFactory(ugi: UserGroupInformation, kerberosEnabled: Boolean) {
+ def wrapHandler(handler: Handler): HandlerWrapper = {
+ new HandlerWrapper {
+ _handler = handler
+
+ override def handle(
+ target: String,
+ baseRequest: Request,
+ request: HttpServletRequest,
+ response: HttpServletResponse): Unit = {
+ try {
+ if (kerberosEnabled) {
+ ugi.doAs(new PrivilegedAction[Unit] {
+ override def run(): Unit = {
+ handler.handle(target, baseRequest, request, response)
+ }
+ })
+ } else {
+ handler.handle(target, baseRequest, request, response)
+ }
+ } finally {
+ AuthenticationFilter.HTTP_CLIENT_USER_NAME.remove()
+ AuthenticationFilter.HTTP_CLIENT_IP_ADDRESS.remove()
+ }
+ }
+
+ override def doStart(): Unit = {
+ super.doStart()
+ handler.start()
+ }
+ }
+ }
+ }
+}
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/ui/JettyServer.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/ui/JettyServer.scala
index 35faee17b..a53b875bd 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/ui/JettyServer.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/ui/JettyServer.scala
@@ -17,16 +17,12 @@
package org.apache.kyuubi.server.ui
-import javax.servlet.http.{HttpServletRequest, HttpServletResponse}
-
-import org.eclipse.jetty.server.{HttpConfiguration, HttpConnectionFactory, Request, Server, ServerConnector}
-import org.eclipse.jetty.server.handler.{ContextHandlerCollection, ErrorHandler, HandlerWrapper}
-import org.eclipse.jetty.servlet.ServletContextHandler
+import org.eclipse.jetty.server.{Handler, HttpConfiguration, HttpConnectionFactory, Server, ServerConnector}
+import org.eclipse.jetty.server.handler.{ContextHandlerCollection, ErrorHandler}
import org.eclipse.jetty.util.component.LifeCycle
import org.eclipse.jetty.util.thread.{QueuedThreadPool, ScheduledExecutorScheduler}
import org.apache.kyuubi.Utils.isWindows
-import org.apache.kyuubi.server.http.authentication.AuthenticationFilter
private[kyuubi] case class JettyServer(
server: Server,
@@ -55,25 +51,9 @@ private[kyuubi] case class JettyServer(
}
def getServerUri: String = connector.getHost + ":" + connector.getLocalPort
- def addHandler(handler: ServletContextHandler): Unit = synchronized {
- val handlerWrapper = new HandlerWrapper {
- override def handle(
- target: String,
- baseRequest: Request,
- request: HttpServletRequest,
- response: HttpServletResponse): Unit = {
- try {
- super.handle(target, baseRequest, request, response)
- } finally {
- AuthenticationFilter.HTTP_CLIENT_USER_NAME.remove()
- AuthenticationFilter.HTTP_CLIENT_IP_ADDRESS.remove()
- }
- }
- }
- handlerWrapper.setHandler(handler)
- rootHandler.addHandler(handlerWrapper)
+ def addHandler(handler: Handler): Unit = synchronized {
+ rootHandler.addHandler(handler)
if (!handler.isStarted) handler.start()
- if (!handlerWrapper.isStarted) handlerWrapper.start()
}
def addStaticHandler(
diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/KyuubiRestAuthenticationSuite.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/KyuubiRestAuthenticationSuite.scala
index cdd920d3f..af68db3eb 100644
--- a/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/KyuubiRestAuthenticationSuite.scala
+++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/KyuubiRestAuthenticationSuite.scala
@@ -19,13 +19,16 @@ package org.apache.kyuubi.operation
import java.util.Base64
import javax.servlet.http.HttpServletResponse
+import javax.ws.rs.client.Entity
+import javax.ws.rs.core.MediaType
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hive.service.rpc.thrift.TProtocolVersion
import org.apache.kyuubi.{KerberizedTestHelper, RestFrontendTestHelper}
import org.apache.kyuubi.config.KyuubiConf
-import org.apache.kyuubi.server.api.v1.SessionOpenCount
+import org.apache.kyuubi.server.api.v1.{SessionOpenCount, SessionOpenRequest}
import org.apache.kyuubi.server.http.authentication.AuthenticationHandler.AUTHORIZATION_HEADER
import org.apache.kyuubi.service.authentication.{UserDefineAuthenticationProviderImpl, WithLdapServer}
@@ -145,4 +148,22 @@ class KyuubiRestAuthenticationSuite extends RestFrontendTestHelper with Kerberiz
val response = webTarget.path("swagger").request().get()
assert(HttpServletResponse.SC_OK == response.getStatus)
}
+
+ test("test with ugi wrapped open session") {
+ UserGroupInformation.loginUserFromKeytab(testPrincipal, testKeytab)
+ val token = generateToken(hostName)
+ val sessionOpenRequest = SessionOpenRequest(
+ TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V11.getValue,
+ "kyuubi",
+ "pass",
+ "localhost",
+ Map.empty[String, String])
+
+ val response = webTarget.path("api/v1/sessions")
+ .request()
+ .header(AUTHORIZATION_HEADER, s"NEGOTIATE $token")
+ .post(Entity.entity(sessionOpenRequest, MediaType.APPLICATION_JSON_TYPE))
+
+ assert(HttpServletResponse.SC_OK == response.getStatus)
+ }
}