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

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


The following commit(s) were added to refs/heads/branch-1.9 by this push:
     new 191d07f08 [KYUUBI #6648] Support to show launch engine log for thrift 
http mode
191d07f08 is described below

commit 191d07f088e5cc3e3cfea46e818885d0e7814e50
Author: Wang, Fei <[email protected]>
AuthorDate: Fri Aug 30 19:29:52 2024 -0700

    [KYUUBI #6648] Support to show launch engine log for thrift http mode
    
    # :mag: Description
    ## Issue References ๐Ÿ”—
    
    This PR is to return the launch engine operation handle for thrift http 
frontend service when opening session.
    
    So that, the kyuubi connection show the launch engine log.
    
    Refer https://github.com/apache/kyuubi/pull/1377
    
    ## Describe Your Solution ๐Ÿ”ง
    
    Please include a summary of the change and which issue is fixed. Please 
also include relevant motivation and context. List any dependencies that are 
required for this change.
    Just return the launch engine op handle for thrift http connection.
    
    ## Types of changes :bookmark:
    
    - [ ] Bugfix (non-breaking change which fixes an issue)
    - [x] 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:
    
    #### Behavior With This Pull Request :tada:
    
    #### Related Unit Tests
    
    ---
    
    # 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 #6648 from turboFei/engine_launch_log.
    
    Closes #6648
    
    6a3540775 [Wang, Fei] nit
    0bd08d208 [Wang, Fei] Support to show launch engine log for thrift http mode
    
    Authored-by: Wang, Fei <[email protected]>
    Signed-off-by: Wang, Fei <[email protected]>
---
 .../kyuubi/server/KyuubiTHttpFrontendService.scala | 50 +++++++++++++++++++++-
 1 file changed, 48 insertions(+), 2 deletions(-)

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 1e289c623..a68448670 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
@@ -18,6 +18,7 @@
 package org.apache.kyuubi.server
 
 import java.net.ServerSocket
+import java.util.Base64
 import java.util.concurrent.{SynchronousQueue, ThreadPoolExecutor, TimeUnit}
 import javax.security.sasl.AuthenticationException
 import javax.servlet.{ServletContextEvent, ServletContextListener}
@@ -33,15 +34,19 @@ import org.eclipse.jetty.util.security.Constraint
 import org.eclipse.jetty.util.ssl.SslContextFactory
 import org.eclipse.jetty.util.thread.ExecutorThreadPool
 
-import org.apache.kyuubi.KyuubiException
+import org.apache.kyuubi.{KyuubiException, KyuubiSQLException}
+import org.apache.kyuubi.cli.Handle
 import org.apache.kyuubi.config.KyuubiConf
 import org.apache.kyuubi.config.KyuubiConf._
+import 
org.apache.kyuubi.config.KyuubiReservedKeys.{KYUUBI_SESSION_ENGINE_LAUNCH_HANDLE_GUID,
 KYUUBI_SESSION_ENGINE_LAUNCH_HANDLE_SECRET, 
KYUUBI_SESSION_ENGINE_LAUNCH_SUPPORT_RESULT}
 import org.apache.kyuubi.metrics.MetricsConstants.{THRIFT_HTTP_CONN_FAIL, 
THRIFT_HTTP_CONN_OPEN, THRIFT_HTTP_CONN_TOTAL}
 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.shaded.hive.service.rpc.thrift.{TCLIService, 
TOpenSessionReq}
+import org.apache.kyuubi.service.TFrontendService.{CURRENT_SERVER_CONTEXT, 
OK_STATUS}
+import org.apache.kyuubi.session.KyuubiSessionImpl
+import org.apache.kyuubi.shaded.hive.service.rpc.thrift.{TCLIService, 
TOpenSessionReq, TOpenSessionResp}
 import org.apache.kyuubi.shaded.thrift.protocol.TBinaryProtocol
 import org.apache.kyuubi.util.NamedThreadFactory
 
@@ -65,6 +70,8 @@ final class KyuubiTHttpFrontendService(
 
   override protected def hadoopConf: Configuration = 
KyuubiServer.getHadoopConf()
 
+  private lazy val defaultFetchSize = 
conf.get(KYUUBI_SERVER_THRIFT_RESULTSET_DEFAULT_FETCH_SIZE)
+
   /**
    * Configure Jetty to serve http requests. Example of a client connection 
URL:
    * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual 
target
@@ -219,6 +226,45 @@ final class KyuubiTHttpFrontendService(
     super.initialize(conf)
   }
 
+  /** Same as KyuubiTBinaryFrontendService, to return launch engine op handle. 
*/
+  override def OpenSession(req: TOpenSessionReq): TOpenSessionResp = {
+    debug(req.toString)
+    info("Client protocol version: " + req.getClient_protocol)
+    val resp = new TOpenSessionResp
+    try {
+      val sessionHandle = getSessionHandle(req, resp)
+
+      val respConfiguration = new java.util.HashMap[String, String]()
+      val launchEngineOp = be.sessionManager.getSession(sessionHandle)
+        .asInstanceOf[KyuubiSessionImpl].launchEngineOp
+
+      val opHandleIdentifier = 
Handle.toTHandleIdentifier(launchEngineOp.getHandle.identifier)
+      respConfiguration.put(
+        KYUUBI_SESSION_ENGINE_LAUNCH_HANDLE_GUID,
+        Base64.getEncoder.encodeToString(opHandleIdentifier.getGuid))
+      respConfiguration.put(
+        KYUUBI_SESSION_ENGINE_LAUNCH_HANDLE_SECRET,
+        Base64.getEncoder.encodeToString(opHandleIdentifier.getSecret))
+
+      respConfiguration.put(KYUUBI_SESSION_ENGINE_LAUNCH_SUPPORT_RESULT, 
true.toString)
+
+      // HIVE-23005(4.0.0), Hive JDBC driver supposes that server always 
returns this conf
+      respConfiguration.put(
+        "hive.server2.thrift.resultset.default.fetch.size",
+        defaultFetchSize.toString)
+
+      resp.setSessionHandle(sessionHandle.toTSessionHandle)
+      resp.setConfiguration(respConfiguration)
+      resp.setStatus(OK_STATUS)
+      
Option(CURRENT_SERVER_CONTEXT.get()).foreach(_.setSessionHandle(sessionHandle))
+    } catch {
+      case e: Exception =>
+        error("Error opening session: ", e)
+        resp.setStatus(KyuubiSQLException.toTStatus(e, verbose = true))
+    }
+    resp
+  }
+
   override def run(): Unit =
     try {
       if (isServer()) {

Reply via email to