You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@kyuubi.apache.org by GitBox <gi...@apache.org> on 2022/03/24 10:21:08 UTC

[GitHub] [incubator-kyuubi] ulysses-you commented on a change in pull request #2122: [KYUUBI #2102] Support to retry the internal thrift request call and add engine liveness probe to enable fast fail before retry

ulysses-you commented on a change in pull request #2122:
URL: https://github.com/apache/incubator-kyuubi/pull/2122#discussion_r834144429



##########
File path: kyuubi-server/src/main/scala/org/apache/kyuubi/client/KyuubiSyncThriftClient.scala
##########
@@ -17,31 +17,75 @@
 
 package org.apache.kyuubi.client
 
+import java.util.concurrent.{ScheduledExecutorService, TimeUnit}
 import java.util.concurrent.locks.ReentrantLock
 
 import scala.collection.JavaConverters._
 
 import org.apache.hive.service.rpc.thrift._
+import org.apache.thrift.TException
 import org.apache.thrift.protocol.{TBinaryProtocol, TProtocol}
 import org.apache.thrift.transport.TSocket
 
-import org.apache.kyuubi.{KyuubiSQLException, Logging}
+import org.apache.kyuubi.{KyuubiSQLException, Logging, Utils}
 import org.apache.kyuubi.config.KyuubiConf
 import org.apache.kyuubi.config.KyuubiConf.{ENGINE_LOGIN_TIMEOUT, ENGINE_REQUEST_TIMEOUT}
 import org.apache.kyuubi.operation.FetchOrientation
 import org.apache.kyuubi.operation.FetchOrientation.FetchOrientation
 import org.apache.kyuubi.service.authentication.PlainSASLHelper
 import org.apache.kyuubi.session.SessionHandle
-import org.apache.kyuubi.util.ThriftUtils
+import org.apache.kyuubi.util.{ThreadUtils, ThriftUtils}
 
-class KyuubiSyncThriftClient private (protocol: TProtocol)
+class KyuubiSyncThriftClient private (
+    protocol: TProtocol,
+    maxAttempts: Int,
+    engineAliveProbeProtocol: Option[TProtocol],
+    engineAliveProbeInterval: Long)
   extends TCLIService.Client(protocol) with Logging {
 
   @volatile private var _remoteSessionHandle: TSessionHandle = _
   @volatile private var _engineId: Option[String] = _
 
   private val lock = new ReentrantLock()
 
+  @volatile private var _aliveProbeSessionHandle: TSessionHandle = _
+  @volatile private var aliveProbeConnectionBroken = false
+  @volatile private var remoteEngineBroken: Boolean = false
+  private val engineAliveProbeClient = engineAliveProbeProtocol.map(new TCLIService.Client(_))
+  private var engineAliveThreadPool: ScheduledExecutorService = _
+
+  private def startEngineAliveProbe(): Unit = {
+    engineAliveThreadPool = ThreadUtils.newDaemonSingleThreadScheduledExecutor(
+      "engine-alive-probe-" + _aliveProbeSessionHandle)
+    val task = new Runnable {
+      override def run(): Unit = {
+        engineAliveProbeClient.foreach { client =>
+          val tGetInfoReq = new TGetInfoReq()
+          tGetInfoReq.setSessionHandle(_aliveProbeSessionHandle)
+          tGetInfoReq.setInfoType(TGetInfoType.CLI_DBMS_VER)
+
+          try {
+            client.GetInfo(tGetInfoReq).getInfoValue.getStringValue
+            aliveProbeConnectionBroken = false
+            remoteEngineBroken = false
+          } catch {
+            case e: Throwable =>
+              if (!aliveProbeConnectionBroken) {
+                error(s"The alive probe connection is broken, assume the engine is not alive", e)
+                aliveProbeConnectionBroken = true
+                remoteEngineBroken = true

Review comment:
       It will be flaky if the probe change to broke as long as fail. shall we also should try with some attempts ?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org