You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@kyuubi.apache.org by "turboFei (via GitHub)" <gi...@apache.org> on 2023/06/20 08:16:56 UTC

[GitHub] [kyuubi] turboFei opened a new pull request, #4984: Support multiple kubernetes contexts and namespaces

turboFei opened a new pull request, #4984:
URL: https://github.com/apache/kyuubi/pull/4984

   <!--
   Thanks for sending a pull request!
   
   Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://kyuubi.readthedocs.io/en/latest/community/CONTRIBUTING.html
     2. If the PR is related to an issue in https://github.com/apache/kyuubi/issues, add '[KYUUBI #XXXX]' in your PR title, e.g., '[KYUUBI #XXXX] Your PR title ...'.
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][KYUUBI #XXXX] Your PR title ...'.
   -->
   
   ### _Why are the changes needed?_
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you add a feature, you can talk about the use case of it.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   
   ### _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.readthedocs.io/en/master/contributing/code/testing.html#running-tests) locally before make a pull request
   


-- 
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


[GitHub] [kyuubi] pan3793 commented on a diff in pull request #4984: [KYUUBI #4843] Support multiple kubernetes contexts and namespaces

Posted by "pan3793 (via GitHub)" <gi...@apache.org>.
pan3793 commented on code in PR #4984:
URL: https://github.com/apache/kyuubi/pull/4984#discussion_r1241709678


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala:
##########
@@ -1106,13 +1132,31 @@ object KyuubiConf {
       .stringConf
       .createOptional
 
+  val KUBERNETES_CONTEXT_LIST: ConfigEntry[Seq[String]] =
+    buildConf("kyuubi.kubernetes.context.list")

Review Comment:
   ```suggestion
       buildConf("kyuubi.kubernetes.context.allow.list")
   ```



##########
kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala:
##########
@@ -1106,13 +1132,31 @@ object KyuubiConf {
       .stringConf
       .createOptional
 
+  val KUBERNETES_CONTEXT_LIST: ConfigEntry[Seq[String]] =
+    buildConf("kyuubi.kubernetes.context.list")
+      .doc("The supported kubernetes context list, if it is empty," +
+        " there is no kubernetes context limitation.")
+      .version("1.8.0")
+      .stringConf
+      .toSequence()
+      .createWithDefault(Nil)
+
   val KUBERNETES_NAMESPACE: ConfigEntry[String] =
     buildConf("kyuubi.kubernetes.namespace")
       .doc("The namespace that will be used for running the kyuubi pods and find engines.")
       .version("1.7.0")
       .stringConf
       .createWithDefault("default")
 
+  val KUBERNETES_NAMESPACE_LIST: ConfigEntry[Seq[String]] =
+    buildConf("kyuubi.kubernetes.namespace.list")

Review Comment:
   ```suggestion
       buildConf("kyuubi.kubernetes.namespace.allow.list")
   ```



-- 
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


[GitHub] [kyuubi] turboFei commented on pull request #4984: [KYUUBI #4843] Support multiple kubernetes contexts and namespaces

Posted by "turboFei (via GitHub)" <gi...@apache.org>.
turboFei commented on PR #4984:
URL: https://github.com/apache/kyuubi/pull/4984#issuecomment-1606873345

   @pan3793 thanks, all comments addressed 
   


-- 
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


[GitHub] [kyuubi] pan3793 commented on a diff in pull request #4984: [KYUUBI #4843] Support multiple kubernetes contexts and namespaces

Posted by "pan3793 (via GitHub)" <gi...@apache.org>.
pan3793 commented on code in PR #4984:
URL: https://github.com/apache/kyuubi/pull/4984#discussion_r1241686692


##########
kyuubi-server/src/main/scala/org/apache/kyuubi/engine/KubernetesApplicationOperation.scala:
##########
@@ -20,94 +20,148 @@ package org.apache.kyuubi.engine
 import java.util.Locale
 import java.util.concurrent.{ConcurrentHashMap, TimeUnit}
 
+import scala.collection.JavaConverters._
+
 import com.google.common.cache.{Cache, CacheBuilder, RemovalNotification}
 import io.fabric8.kubernetes.api.model.Pod
 import io.fabric8.kubernetes.client.KubernetesClient
 import io.fabric8.kubernetes.client.informers.{ResourceEventHandler, SharedIndexInformer}
 
-import org.apache.kyuubi.{Logging, Utils}
+import org.apache.kyuubi.{KyuubiException, Logging, Utils}
 import org.apache.kyuubi.config.KyuubiConf
 import org.apache.kyuubi.engine.ApplicationState.{isTerminated, ApplicationState, FAILED, FINISHED, NOT_FOUND, PENDING, RUNNING, UNKNOWN}
 import org.apache.kyuubi.engine.KubernetesApplicationOperation.{toApplicationState, LABEL_KYUUBI_UNIQUE_KEY, SPARK_APP_ID_LABEL}
 import org.apache.kyuubi.util.KubernetesUtils
 
 class KubernetesApplicationOperation extends ApplicationOperation with Logging {
 
-  @volatile
-  private var kubernetesClient: KubernetesClient = _
-  private var enginePodInformer: SharedIndexInformer[Pod] = _
+  private val kubernetesClients: ConcurrentHashMap[KubernetesInfo, KubernetesClient] =
+    new ConcurrentHashMap[KubernetesInfo, KubernetesClient]
+  private val enginePodInformers: ConcurrentHashMap[KubernetesInfo, SharedIndexInformer[Pod]] =
+    new ConcurrentHashMap[KubernetesInfo, SharedIndexInformer[Pod]]
+
+  private var supportedContexts: Seq[String] = Seq.empty
+  private var supportedNamespaces: Seq[String] = Seq.empty
+
   private var submitTimeout: Long = _
+  private var kyuubiConf: KyuubiConf = _
 
   // key is kyuubi_unique_key
   private val appInfoStore: ConcurrentHashMap[String, ApplicationInfo] =
     new ConcurrentHashMap[String, ApplicationInfo]
   // key is kyuubi_unique_key
   private var cleanupTerminatedAppInfoTrigger: Cache[String, ApplicationState] = _
 
-  override def initialize(conf: KyuubiConf): Unit = {
-    info("Start initializing Kubernetes Client.")
-    kubernetesClient = KubernetesUtils.buildKubernetesClient(conf) match {
+  private def getOrCreateKubernetesClient(kubernetesInfo: KubernetesInfo): KubernetesClient = {
+    val context = kubernetesInfo.context
+    val namespace = kubernetesInfo.namespace
+
+    if (supportedContexts.nonEmpty && !supportedContexts.contains(context)) {
+      throw new KyuubiException(
+        s"Kubernetes context $context is not in the support list[$supportedContexts]")
+    }
+
+    if (supportedNamespaces.nonEmpty && !supportedNamespaces.contains(namespace)) {
+      throw new KyuubiException(
+        s"Kubernetes namespace $namespace is not in the support list[$supportedNamespaces]")
+    }
+
+    var kubernetesClient = kubernetesClients.get(kubernetesInfo)
+    if (kubernetesClient == null) {
+      synchronized {
+        kubernetesClient = kubernetesClients.get(kubernetesInfo)
+        if (kubernetesClient == null) {
+          kubernetesClients.put(kubernetesInfo, buildKubernetesClient(kubernetesInfo))
+        }
+      }
+    }
+    kubernetesClient
+  }
+
+  private def buildKubernetesClient(kubernetesInfo: KubernetesInfo): KubernetesClient = {
+    val kubernetesConf =
+      kyuubiConf.getKubernetesConf(kubernetesInfo.context, kubernetesInfo.namespace)
+    KubernetesUtils.buildKubernetesClient(kubernetesConf) match {
       case Some(client) =>
-        info(s"Initialized Kubernetes Client connect to: ${client.getMasterUrl}")
-        submitTimeout = conf.get(KyuubiConf.ENGINE_KUBERNETES_SUBMIT_TIMEOUT)
-        // Disable resync, see https://github.com/fabric8io/kubernetes-client/discussions/5015
-        enginePodInformer = client.pods()
+        info(s"[$kubernetesInfo] Initialized Kubernetes Client connect to: ${client.getMasterUrl}")
+        val enginePodInformer = client.pods()
           .withLabel(LABEL_KYUUBI_UNIQUE_KEY)
           .inform(new SparkEnginePodEventHandler)
-        info("Start Kubernetes Client Informer.")
-        // Defer cleaning terminated application information
-        val retainPeriod = conf.get(KyuubiConf.KUBERNETES_TERMINATED_APPLICATION_RETAIN_PERIOD)
-        cleanupTerminatedAppInfoTrigger = CacheBuilder.newBuilder()
-          .expireAfterWrite(retainPeriod, TimeUnit.MILLISECONDS)
-          .removalListener((notification: RemovalNotification[String, ApplicationState]) => {
-            Option(appInfoStore.remove(notification.getKey)).foreach { removed =>
-              info(s"Remove terminated application ${removed.id} with " +
-                s"tag ${notification.getKey} and state ${removed.state}")
-            }
-          })
-          .build()
+        info(s"[$kubernetesInfo] Start Kubernetes Client Informer.")
+        enginePodInformers.putIfAbsent(kubernetesInfo, enginePodInformer)

Review Comment:
   in which case non-absent will happen?



-- 
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


[GitHub] [kyuubi] turboFei commented on a diff in pull request #4984: [KYUUBI #4843] Support multiple kubernetes contexts and namespaces

Posted by "turboFei (via GitHub)" <gi...@apache.org>.
turboFei commented on code in PR #4984:
URL: https://github.com/apache/kyuubi/pull/4984#discussion_r1237912054


##########
kyuubi-server/src/main/scala/org/apache/kyuubi/server/metadata/api/Metadata.scala:
##########
@@ -66,6 +68,8 @@ case class Metadata(
     createTime: Long = 0L,
     engineType: String = null,
     clusterManager: Option[String] = None,
+    kubernetesContext: Option[String] = None,
+    kubernetesNamespace: Option[String] = None,

Review Comment:
   Good idea



-- 
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


[GitHub] [kyuubi] turboFei closed pull request #4984: [KYUUBI #4843] Support multiple kubernetes contexts and namespaces

Posted by "turboFei (via GitHub)" <gi...@apache.org>.
turboFei closed pull request #4984: [KYUUBI #4843] Support multiple kubernetes contexts and namespaces
URL: https://github.com/apache/kyuubi/pull/4984


-- 
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


[GitHub] [kyuubi] turboFei commented on pull request #4984: [KYUUBI #4843] Support multiple kubernetes contexts and namespaces

Posted by "turboFei (via GitHub)" <gi...@apache.org>.
turboFei commented on PR #4984:
URL: https://github.com/apache/kyuubi/pull/4984#issuecomment-1600718712

    test passed @pan3793 


-- 
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


[GitHub] [kyuubi] pan3793 commented on a diff in pull request #4984: [KYUUBI #4843] Support multiple kubernetes contexts and namespaces

Posted by "pan3793 (via GitHub)" <gi...@apache.org>.
pan3793 commented on code in PR #4984:
URL: https://github.com/apache/kyuubi/pull/4984#discussion_r1241676393


##########
kyuubi-server/src/main/scala/org/apache/kyuubi/engine/KubernetesApplicationOperation.scala:
##########
@@ -20,94 +20,148 @@ package org.apache.kyuubi.engine
 import java.util.Locale
 import java.util.concurrent.{ConcurrentHashMap, TimeUnit}
 
+import scala.collection.JavaConverters._
+
 import com.google.common.cache.{Cache, CacheBuilder, RemovalNotification}
 import io.fabric8.kubernetes.api.model.Pod
 import io.fabric8.kubernetes.client.KubernetesClient
 import io.fabric8.kubernetes.client.informers.{ResourceEventHandler, SharedIndexInformer}
 
-import org.apache.kyuubi.{Logging, Utils}
+import org.apache.kyuubi.{KyuubiException, Logging, Utils}
 import org.apache.kyuubi.config.KyuubiConf
 import org.apache.kyuubi.engine.ApplicationState.{isTerminated, ApplicationState, FAILED, FINISHED, NOT_FOUND, PENDING, RUNNING, UNKNOWN}
 import org.apache.kyuubi.engine.KubernetesApplicationOperation.{toApplicationState, LABEL_KYUUBI_UNIQUE_KEY, SPARK_APP_ID_LABEL}
 import org.apache.kyuubi.util.KubernetesUtils
 
 class KubernetesApplicationOperation extends ApplicationOperation with Logging {
 
-  @volatile
-  private var kubernetesClient: KubernetesClient = _
-  private var enginePodInformer: SharedIndexInformer[Pod] = _
+  private val kubernetesClients: ConcurrentHashMap[KubernetesInfo, KubernetesClient] =
+    new ConcurrentHashMap[KubernetesInfo, KubernetesClient]
+  private val enginePodInformers: ConcurrentHashMap[KubernetesInfo, SharedIndexInformer[Pod]] =
+    new ConcurrentHashMap[KubernetesInfo, SharedIndexInformer[Pod]]
+
+  private var supportedContexts: Seq[String] = Seq.empty
+  private var supportedNamespaces: Seq[String] = Seq.empty
+
   private var submitTimeout: Long = _
+  private var kyuubiConf: KyuubiConf = _
 
   // key is kyuubi_unique_key
   private val appInfoStore: ConcurrentHashMap[String, ApplicationInfo] =
     new ConcurrentHashMap[String, ApplicationInfo]
   // key is kyuubi_unique_key
   private var cleanupTerminatedAppInfoTrigger: Cache[String, ApplicationState] = _
 
-  override def initialize(conf: KyuubiConf): Unit = {
-    info("Start initializing Kubernetes Client.")
-    kubernetesClient = KubernetesUtils.buildKubernetesClient(conf) match {
+  private def getOrCreateKubernetesClient(kubernetesInfo: KubernetesInfo): KubernetesClient = {
+    val context = kubernetesInfo.context
+    val namespace = kubernetesInfo.namespace
+
+    if (supportedContexts.nonEmpty && !supportedContexts.contains(context)) {
+      throw new KyuubiException(
+        s"Kubernetes context $context is not in the support list[$supportedContexts]")
+    }
+
+    if (supportedNamespaces.nonEmpty && !supportedNamespaces.contains(namespace)) {
+      throw new KyuubiException(
+        s"Kubernetes namespace $namespace is not in the support list[$supportedNamespaces]")
+    }
+
+    var kubernetesClient = kubernetesClients.get(kubernetesInfo)
+    if (kubernetesClient == null) {

Review Comment:
   computeIfAbsent?



-- 
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


[GitHub] [kyuubi] pan3793 commented on a diff in pull request #4984: [KYUUBI #4843] Support multiple kubernetes contexts and namespaces

Posted by "pan3793 (via GitHub)" <gi...@apache.org>.
pan3793 commented on code in PR #4984:
URL: https://github.com/apache/kyuubi/pull/4984#discussion_r1237302897


##########
kyuubi-server/src/main/scala/org/apache/kyuubi/server/metadata/api/Metadata.scala:
##########
@@ -66,6 +68,8 @@ case class Metadata(
     createTime: Long = 0L,
     engineType: String = null,
     clusterManager: Option[String] = None,
+    kubernetesContext: Option[String] = None,
+    kubernetesNamespace: Option[String] = None,

Review Comment:
   will this cause metadata DDL change? I prefer to store them in requestConf, and make them as method instead of `val`.



-- 
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


[GitHub] [kyuubi] turboFei commented on pull request #4984: [KYUUBI #4843] Support multiple kubernetes contexts and namespaces

Posted by "turboFei (via GitHub)" <gi...@apache.org>.
turboFei commented on PR #4984:
URL: https://github.com/apache/kyuubi/pull/4984#issuecomment-1606906752

   thanks a lot @pan3793 , merging to master


-- 
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


[GitHub] [kyuubi] codecov-commenter commented on pull request #4984: Support multiple kubernetes contexts and namespaces

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #4984:
URL: https://github.com/apache/kyuubi/pull/4984#issuecomment-1600261798

   ## [Codecov](https://app.codecov.io/gh/apache/kyuubi/pull/4984?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report
   > Merging [#4984](https://app.codecov.io/gh/apache/kyuubi/pull/4984?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (89656f4) into [master](https://app.codecov.io/gh/apache/kyuubi/commit/13f11c7ab4d99894eafc6dfa9fba7228016632db?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (13f11c7) will **not change** coverage.
   > The diff coverage is `0.00%`.
   
   > :exclamation: Current head 89656f4 differs from pull request most recent head aa568aa. Consider uploading reports for the commit aa568aa to get more accurate results
   
   ```diff
   @@          Coverage Diff           @@
   ##           master   #4984   +/-   ##
   ======================================
     Coverage    0.00%   0.00%           
   ======================================
     Files         561     561           
     Lines       30953   31037   +84     
     Branches     4063    4071    +8     
   ======================================
   - Misses      30953   31037   +84     
   ```
   
   
   | [Impacted Files](https://app.codecov.io/gh/apache/kyuubi/pull/4984?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Coverage Δ | |
   |---|---|---|
   | [...in/scala/org/apache/kyuubi/config/KyuubiConf.scala](https://app.codecov.io/gh/apache/kyuubi/pull/4984?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-a3l1dWJpLWNvbW1vbi9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9jb25maWcvS3l1dWJpQ29uZi5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...rg/apache/kyuubi/engine/ApplicationOperation.scala](https://app.codecov.io/gh/apache/kyuubi/pull/4984?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9lbmdpbmUvQXBwbGljYXRpb25PcGVyYXRpb24uc2NhbGE=) | `0.00% <0.00%> (ø)` | |
   | [...ain/scala/org/apache/kyuubi/engine/EngineRef.scala](https://app.codecov.io/gh/apache/kyuubi/pull/4984?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9lbmdpbmUvRW5naW5lUmVmLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [...apache/kyuubi/engine/JpsApplicationOperation.scala](https://app.codecov.io/gh/apache/kyuubi/pull/4984?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9lbmdpbmUvSnBzQXBwbGljYXRpb25PcGVyYXRpb24uc2NhbGE=) | `0.00% <0.00%> (ø)` | |
   | [...kyuubi/engine/KubernetesApplicationOperation.scala](https://app.codecov.io/gh/apache/kyuubi/pull/4984?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9lbmdpbmUvS3ViZXJuZXRlc0FwcGxpY2F0aW9uT3BlcmF0aW9uLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [...pache/kyuubi/engine/KyuubiApplicationManager.scala](https://app.codecov.io/gh/apache/kyuubi/pull/4984?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9lbmdpbmUvS3l1dWJpQXBwbGljYXRpb25NYW5hZ2VyLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [...n/scala/org/apache/kyuubi/engine/ProcBuilder.scala](https://app.codecov.io/gh/apache/kyuubi/pull/4984?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9lbmdpbmUvUHJvY0J1aWxkZXIuc2NhbGE=) | `0.00% <0.00%> (ø)` | |
   | [...pache/kyuubi/engine/YarnApplicationOperation.scala](https://app.codecov.io/gh/apache/kyuubi/pull/4984?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9lbmdpbmUvWWFybkFwcGxpY2F0aW9uT3BlcmF0aW9uLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [...kyuubi/engine/spark/SparkBatchProcessBuilder.scala](https://app.codecov.io/gh/apache/kyuubi/pull/4984?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9lbmdpbmUvc3BhcmsvU3BhcmtCYXRjaFByb2Nlc3NCdWlsZGVyLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [...ache/kyuubi/engine/spark/SparkProcessBuilder.scala](https://app.codecov.io/gh/apache/kyuubi/pull/4984?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9lbmdpbmUvc3BhcmsvU3BhcmtQcm9jZXNzQnVpbGRlci5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | ... and [3 more](https://app.codecov.io/gh/apache/kyuubi/pull/4984?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | |
   
   ... and [1 file with indirect coverage changes](https://app.codecov.io/gh/apache/kyuubi/pull/4984/indirect-changes?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)
   


-- 
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


[GitHub] [kyuubi] pan3793 commented on a diff in pull request #4984: [KYUUBI #4843] Support multiple kubernetes contexts and namespaces

Posted by "pan3793 (via GitHub)" <gi...@apache.org>.
pan3793 commented on code in PR #4984:
URL: https://github.com/apache/kyuubi/pull/4984#discussion_r1237301122


##########
kyuubi-server/src/main/scala/org/apache/kyuubi/engine/ProcBuilder.scala:
##########
@@ -341,6 +341,12 @@ trait ProcBuilder {
 
   def clusterManager(): Option[String] = None
 
+  def kubernetesContext(): Option[String] = None
+
+  def kubernetesNamespace(): Option[String] = None

Review Comment:
   That's too crude, the ProcBuilder's API should be generic, otherwise as time going, lots of `kubernetesXYZs` methods will be added in the future.



-- 
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


[GitHub] [kyuubi] turboFei commented on a diff in pull request #4984: [KYUUBI #4843] Support multiple kubernetes contexts and namespaces

Posted by "turboFei (via GitHub)" <gi...@apache.org>.
turboFei commented on code in PR #4984:
URL: https://github.com/apache/kyuubi/pull/4984#discussion_r1238455600


##########
kyuubi-server/src/main/scala/org/apache/kyuubi/session/KyuubiBatchSession.scala:
##########
@@ -154,7 +160,7 @@ class KyuubiBatchSession(
         resource = resource,
         className = className,
         requestName = name.orNull,
-        requestConf = optimizedConf,
+        requestConf = optimizedConf ++ kubernetesInfo, // save the kubernetes info into request conf

Review Comment:
   done @pan3793 



-- 
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


[GitHub] [kyuubi] turboFei commented on pull request #4984: [KYUUBI #4843] Support multiple kubernetes contexts and namespaces

Posted by "turboFei (via GitHub)" <gi...@apache.org>.
turboFei commented on PR #4984:
URL: https://github.com/apache/kyuubi/pull/4984#issuecomment-1602645444

   ```
   DockerizedZkServiceDiscoverySuite:
   - publish instance to discovery service
   - KYUUBI #304: Stop engine service gracefully when related node is deleted
   - distribute lock *** FAILED ***
     Expected exception org.apache.kyuubi.KyuubiSQLException to be thrown, but no exception was thrown (DiscoveryClientTests.scala:147)
   ```


-- 
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