You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/09/05 09:56:48 UTC

[GitHub] [spark] zheniantoushipashi opened a new pull request, #37802: [SPARK-40350][Kubernetes] Use spark config to configure the parameters of volcano podgroup

zheniantoushipashi opened a new pull request, #37802:
URL: https://github.com/apache/spark/pull/37802

   What changes were proposed in this 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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zheniantoushipashi commented on pull request #37802: [SPARK-40350][Kubernetes] Use spark config to configure the parameters of volcano podgroup

Posted by GitBox <gi...@apache.org>.
zheniantoushipashi commented on PR #37802:
URL: https://github.com/apache/spark/pull/37802#issuecomment-1238949681

   @Yikun    @dongjoon-hyun @holdenk   


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Yikun commented on a diff in pull request #37802: [SPARK-40350][Kubernetes] Use spark config to configure the parameters of volcano podgroup

Posted by GitBox <gi...@apache.org>.
Yikun commented on code in PR #37802:
URL: https://github.com/apache/spark/pull/37802#discussion_r963695606


##########
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStep.scala:
##########
@@ -54,11 +56,33 @@ private[spark] class VolcanoFeatureStep extends KubernetesDriverCustomFeatureCon
     metadata.setName(podGroupName)
     metadata.setNamespace(namespace)
     pg.setMetadata(metadata)
-
     var spec = pg.getSpec
     if (spec == null) spec = new PodGroupSpec
+    val queue = kubernetesConf.getOption(POD_GROUP_SPEC_QUEUE)
+    if (queue.isDefined) {
+      spec.setQueue(queue.get)
+    }
+    val minResourceCPU = kubernetesConf.getOption(POD_GROUP_SPEC_MIN_RESOURCE_CPU)
+    val minResourceMemory = kubernetesConf.getOption(POD_GROUP_SPEC_MIN_RESOURCE_MEMORY)
+    if (minResourceCPU.isDefined || minResourceMemory.isDefined) {
+      val minResources = new util.HashMap[String, Quantity]
+      if (minResourceCPU.isDefined) {
+        minResources.put("cpu", new Quantity(minResourceCPU.get))
+      }
+      if (minResourceMemory.isDefined) {
+        minResources.put("memory", new Quantity(minResourceMemory.get))
+      }
+      spec.setMinResources(minResources)

Review Comment:
   This will force to overwrite existing MinResources and ignore existing template once one of configurations set 
   
   



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Yikun commented on a diff in pull request #37802: [SPARK-40350][Kubernetes] Use spark config to configure the parameters of volcano podgroup

Posted by GitBox <gi...@apache.org>.
Yikun commented on code in PR #37802:
URL: https://github.com/apache/spark/pull/37802#discussion_r963695606


##########
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStep.scala:
##########
@@ -54,11 +56,33 @@ private[spark] class VolcanoFeatureStep extends KubernetesDriverCustomFeatureCon
     metadata.setName(podGroupName)
     metadata.setNamespace(namespace)
     pg.setMetadata(metadata)
-
     var spec = pg.getSpec
     if (spec == null) spec = new PodGroupSpec
+    val queue = kubernetesConf.getOption(POD_GROUP_SPEC_QUEUE)
+    if (queue.isDefined) {
+      spec.setQueue(queue.get)
+    }
+    val minResourceCPU = kubernetesConf.getOption(POD_GROUP_SPEC_MIN_RESOURCE_CPU)
+    val minResourceMemory = kubernetesConf.getOption(POD_GROUP_SPEC_MIN_RESOURCE_MEMORY)
+    if (minResourceCPU.isDefined || minResourceMemory.isDefined) {
+      val minResources = new util.HashMap[String, Quantity]
+      if (minResourceCPU.isDefined) {
+        minResources.put("cpu", new Quantity(minResourceCPU.get))
+      }
+      if (minResourceMemory.isDefined) {
+        minResources.put("memory", new Quantity(minResourceMemory.get))
+      }
+      spec.setMinResources(minResources)

Review Comment:
   This will force to overwrite existing MinResources and ignore existing template if one of configurations is set.
   
   



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Yikun commented on pull request #37802: [SPARK-40350][Kubernetes] Use spark config to configure the parameters of volcano podgroup

Posted by GitBox <gi...@apache.org>.
Yikun commented on PR #37802:
URL: https://github.com/apache/spark/pull/37802#issuecomment-1247701070

   > Is there a way to specify a queue dynamically when using volcano to submit spark applicaiton?
   
   @zheniantoushipashi Currnently, as you know we can only specify volcano queue by [using PodGroupTemplate](https://github.com/apache/spark/blob/master/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/VolcanoTestsSuite.scala#L255-L260). @dongjoon-hyun meant Spark + Yunikorn can set queue by setting label but volcano couldn't. 
   
   Actually, there are mainly two style to specify scheduler hint in common scheduler:
   - Using annotation/label
   - Using a separate CRD
   
   Because in K8S scheduling, the queue is only one of the capabilities, cooporate with other PodGroup information. If it is also placed in the label or annotation, will be very complicated, it is not the same way as [K8S annotation suggests](https://kubernetes.io/docs/concepts/overview/working-with-objects/annotations/#attaching-metadata-to-objects).
   
   The queue and other parts of the PodGroup are usually considered at the same time, so the queue is used as a field of the PodGroup in Volcano. Users can configure all batch scheduling through the pod group CRD.
   
   Therefore, you might want to `use PodGroupTemplate to cover the whole scene, and then increase flexibility through configuration items` in your own downstream version at current.
   
   Also cc @william-wang
   
   
   


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #37802: [SPARK-40350][Kubernetes] Use spark config to configure the parameters of volcano podgroup

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on PR #37802:
URL: https://github.com/apache/spark/pull/37802#issuecomment-1237518978

   Can one of the admins verify this patch?


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zheniantoushipashi commented on pull request #37802: [SPARK-40350][Kubernetes] Use spark config to configure the parameters of volcano podgroup

Posted by GitBox <gi...@apache.org>.
zheniantoushipashi commented on PR #37802:
URL: https://github.com/apache/spark/pull/37802#issuecomment-1238949312

   I studied your previous discussions, but  i  still think the change i pull request  is  useful,   i   investigative  this  document https://docs.google.com/document/d/1xgQGRpaHQX6-QH_J9YV2C2Dh6RpXefUpLM7KGkzL6Fg/edit#heading=h.1quyr1r2kr5n  ,  i known  spark will  support more  scheduler in the future,   but  the configuration i  add in this pr is just  for   volcano scheduler ,  and the change is  just in the   VolcanoFeatureStep.scala , and the configuration  prefixed with  spark.kubernetes.scheduler.volcano.podGroup ,  not  a  global  spark configuration for  all the  scheduler,  
   
   We can arbitrarily create queues in volcano,  and  In our scenario,   we need to dynamically specify the volcano queue,   our users can dynamically use different queues ,   if we must create a  template file Just to dynamically use the queue I created, it is very  inconvenient, i  must add some code to  create a file  or i  must use configmap to mount the  a new template file and restart the k8s  application,     it is no cool,   
   
   
   i  think  the  template file and the configuration is not conflict,  
   
   ![image](https://user-images.githubusercontent.com/3105102/188801328-d78f9ca4-23c7-43c0-8fcd-c052bff8ba23.png)
   
   
   spark driver pod and executor pod can  use  template define  the pod resource,  and spark configuration can override the template file,  I think this usage is reasonable。 
   
   
     


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Yikun commented on a diff in pull request #37802: [SPARK-40350][Kubernetes] Use spark config to configure the parameters of volcano podgroup

Posted by GitBox <gi...@apache.org>.
Yikun commented on code in PR #37802:
URL: https://github.com/apache/spark/pull/37802#discussion_r963695606


##########
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStep.scala:
##########
@@ -54,11 +56,33 @@ private[spark] class VolcanoFeatureStep extends KubernetesDriverCustomFeatureCon
     metadata.setName(podGroupName)
     metadata.setNamespace(namespace)
     pg.setMetadata(metadata)
-
     var spec = pg.getSpec
     if (spec == null) spec = new PodGroupSpec
+    val queue = kubernetesConf.getOption(POD_GROUP_SPEC_QUEUE)
+    if (queue.isDefined) {
+      spec.setQueue(queue.get)
+    }
+    val minResourceCPU = kubernetesConf.getOption(POD_GROUP_SPEC_MIN_RESOURCE_CPU)
+    val minResourceMemory = kubernetesConf.getOption(POD_GROUP_SPEC_MIN_RESOURCE_MEMORY)
+    if (minResourceCPU.isDefined || minResourceMemory.isDefined) {
+      val minResources = new util.HashMap[String, Quantity]
+      if (minResourceCPU.isDefined) {
+        minResources.put("cpu", new Quantity(minResourceCPU.get))
+      }
+      if (minResourceMemory.isDefined) {
+        minResources.put("memory", new Quantity(minResourceMemory.get))
+      }
+      spec.setMinResources(minResources)

Review Comment:
   This will force to overwrite existing MinResources and ignore existing template once one of configurations is set 
   
   



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on pull request #37802: [SPARK-40350][Kubernetes] Use spark config to configure the parameters of volcano podgroup

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on PR #37802:
URL: https://github.com/apache/spark/pull/37802#issuecomment-1242559032

   Let me close this PR for now. We can continue our discussion on this PR.


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Yikun commented on pull request #37802: [SPARK-40350][Kubernetes] Use spark config to configure the parameters of volcano podgroup

Posted by GitBox <gi...@apache.org>.
Yikun commented on PR #37802:
URL: https://github.com/apache/spark/pull/37802#issuecomment-1239590984

   @zheniantoushipashi Much thanks for you feedback. Most of spark + volcano users who I have come into contact with before, they also configure each job through configuration items.
   
   There is no doubt that `PodGroupTemplate`'s mechanism will be greatly improved for generality. Another previous consideration is to **reduce the frequent addition of configuration items**.
   
   But it is worth noting that `PodGroupTemplate` is different from `PodTemplate`: the CRD of `Pod` has a lot of options, but the CRD of `PodGroup`, especially the commonly used configurations, is relatively fixed (Therefore, configuration items will not be added frequently). Commonly used are `queue`, `minResource`, `priority`. BTW, please note that `minMember` is a bit special, because `minMember` is set to `1` to monitor the driver/job life cycle (see more in https://github.com/volcano-sh/volcano/pull/2038), then to ensure the accuracy of scheduling. Therefore, this parameter should not be set arbitrarily in spark + volcano case.
   
   Therefore, I personally think it is reasonable to use `PodGroupTemplate` to cover the whole scene, and then increase flexibility through configuration items. But we still need more voice from spark community before this.


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zheniantoushipashi commented on pull request #37802: [SPARK-40350][Kubernetes] Use spark config to configure the parameters of volcano podgroup

Posted by GitBox <gi...@apache.org>.
zheniantoushipashi commented on PR #37802:
URL: https://github.com/apache/spark/pull/37802#issuecomment-1247652726

   sorry,   actually  I mean  specifying queues,   Is there a way to specify a queue dynamically  when using volcano to submit spark  applicaiton ?  do you mean  that we should follow the YuniKorn way to implement the dynamically specified queue of volcano  ? 


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on pull request #37802: [SPARK-40350][Kubernetes] Use spark config to configure the parameters of volcano podgroup

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on PR #37802:
URL: https://github.com/apache/spark/pull/37802#issuecomment-1246490993

   > You can't imagine a resource manager compared to yarn, which can't dynamically specify queues when starting application
   
   This is wrong because Apache YuniKorn is able to dynamically specify queues as you see https://github.com/apache/spark/pull/37877 via  the existing Apache Spark labels and annotations without any modification, @zheniantoushipashi .
   
   > Because in our scenario, if the queue used by the scheduler cannot be dynamically configured, the scheduler is basically useless. 
   
   That doesn't sound like a common valid scenario. If you need to create a queue per a single Spark job, what is the meaning of scheduling?


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Yikun commented on a diff in pull request #37802: [SPARK-40350][Kubernetes] Use spark config to configure the parameters of volcano podgroup

Posted by GitBox <gi...@apache.org>.
Yikun commented on code in PR #37802:
URL: https://github.com/apache/spark/pull/37802#discussion_r963695606


##########
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStep.scala:
##########
@@ -54,11 +56,33 @@ private[spark] class VolcanoFeatureStep extends KubernetesDriverCustomFeatureCon
     metadata.setName(podGroupName)
     metadata.setNamespace(namespace)
     pg.setMetadata(metadata)
-
     var spec = pg.getSpec
     if (spec == null) spec = new PodGroupSpec
+    val queue = kubernetesConf.getOption(POD_GROUP_SPEC_QUEUE)
+    if (queue.isDefined) {
+      spec.setQueue(queue.get)
+    }
+    val minResourceCPU = kubernetesConf.getOption(POD_GROUP_SPEC_MIN_RESOURCE_CPU)
+    val minResourceMemory = kubernetesConf.getOption(POD_GROUP_SPEC_MIN_RESOURCE_MEMORY)
+    if (minResourceCPU.isDefined || minResourceMemory.isDefined) {
+      val minResources = new util.HashMap[String, Quantity]
+      if (minResourceCPU.isDefined) {
+        minResources.put("cpu", new Quantity(minResourceCPU.get))
+      }
+      if (minResourceMemory.isDefined) {
+        minResources.put("memory", new Quantity(minResourceMemory.get))
+      }
+      spec.setMinResources(minResources)

Review Comment:
   This will force to replace existing MinResources in template and ignore existing template
   
   



##########
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStep.scala:
##########
@@ -54,11 +56,33 @@ private[spark] class VolcanoFeatureStep extends KubernetesDriverCustomFeatureCon
     metadata.setName(podGroupName)
     metadata.setNamespace(namespace)
     pg.setMetadata(metadata)
-
     var spec = pg.getSpec
     if (spec == null) spec = new PodGroupSpec
+    val queue = kubernetesConf.getOption(POD_GROUP_SPEC_QUEUE)

Review Comment:
   ```scala
   val pg = template.map(client.podGroups.load(_).get).getOrElse(new PodGroup())
   val builder = new PodGroupBuilder(pg)
     .editOrNewMetadata()
       .withName(podGroupName)
       .withNamespace(namespace)
     .endMetadata()
     .editOrNewSpec()
     .endSpec()
   
   if (xxx) {
     builder.editOrNewSpec().endSpec()
   }
   
   return Seq(builder.build())
   ```
   
   If we decided to add parameters support, the builder style might more stable and easy to maintainance.



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Yikun commented on a diff in pull request #37802: [SPARK-40350][Kubernetes] Use spark config to configure the parameters of volcano podgroup

Posted by GitBox <gi...@apache.org>.
Yikun commented on code in PR #37802:
URL: https://github.com/apache/spark/pull/37802#discussion_r963695606


##########
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStep.scala:
##########
@@ -54,11 +56,33 @@ private[spark] class VolcanoFeatureStep extends KubernetesDriverCustomFeatureCon
     metadata.setName(podGroupName)
     metadata.setNamespace(namespace)
     pg.setMetadata(metadata)
-
     var spec = pg.getSpec
     if (spec == null) spec = new PodGroupSpec
+    val queue = kubernetesConf.getOption(POD_GROUP_SPEC_QUEUE)
+    if (queue.isDefined) {
+      spec.setQueue(queue.get)
+    }
+    val minResourceCPU = kubernetesConf.getOption(POD_GROUP_SPEC_MIN_RESOURCE_CPU)
+    val minResourceMemory = kubernetesConf.getOption(POD_GROUP_SPEC_MIN_RESOURCE_MEMORY)
+    if (minResourceCPU.isDefined || minResourceMemory.isDefined) {
+      val minResources = new util.HashMap[String, Quantity]
+      if (minResourceCPU.isDefined) {
+        minResources.put("cpu", new Quantity(minResourceCPU.get))
+      }
+      if (minResourceMemory.isDefined) {
+        minResources.put("memory", new Quantity(minResourceMemory.get))
+      }
+      spec.setMinResources(minResources)

Review Comment:
   This will force to overwrite existing MinResources once one of configurations set and ignore existing template
   
   



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zheniantoushipashi commented on a diff in pull request #37802: [SPARK-40350][Kubernetes] Use spark config to configure the parameters of volcano podgroup

Posted by GitBox <gi...@apache.org>.
zheniantoushipashi commented on code in PR #37802:
URL: https://github.com/apache/spark/pull/37802#discussion_r964420415


##########
resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStep.scala:
##########
@@ -54,11 +56,33 @@ private[spark] class VolcanoFeatureStep extends KubernetesDriverCustomFeatureCon
     metadata.setName(podGroupName)
     metadata.setNamespace(namespace)
     pg.setMetadata(metadata)
-
     var spec = pg.getSpec
     if (spec == null) spec = new PodGroupSpec
+    val queue = kubernetesConf.getOption(POD_GROUP_SPEC_QUEUE)

Review Comment:
   I can refactor this, if my opinion is accepted,



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun closed pull request #37802: [SPARK-40350][Kubernetes] Use spark config to configure the parameters of volcano podgroup

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun closed pull request #37802: [SPARK-40350][Kubernetes] Use spark config to configure the parameters of volcano podgroup
URL: https://github.com/apache/spark/pull/37802


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zheniantoushipashi commented on pull request #37802: [SPARK-40350][Kubernetes] Use spark config to configure the parameters of volcano podgroup

Posted by GitBox <gi...@apache.org>.
zheniantoushipashi commented on PR #37802:
URL: https://github.com/apache/spark/pull/37802#issuecomment-1247652860

   @dongjoon-hyun  


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zheniantoushipashi commented on pull request #37802: [SPARK-40350][Kubernetes] Use spark config to configure the parameters of volcano podgroup

Posted by GitBox <gi...@apache.org>.
zheniantoushipashi commented on PR #37802:
URL: https://github.com/apache/spark/pull/37802#issuecomment-1246472773

   @Yikun @dongjoon-hyun   ok,   If in principle we can't wrap volcano's configuration in spark configuration,  then I can only implement this feature in our own version,  Because in our scenario, if the queue used by the scheduler cannot be dynamically configured, the scheduler is basically useless.  You can't imagine a resource manager compared to yarn, which can't dynamically specify queues when starting application


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on pull request #37802: [SPARK-40350][Kubernetes] Use spark config to configure the parameters of volcano podgroup

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on PR #37802:
URL: https://github.com/apache/spark/pull/37802#issuecomment-1246495348

   Let's distinguish **creating queues** from **specifying queues**. They are not the same.


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org