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 2021/10/12 19:17:30 UTC

[GitHub] [spark] holdenk opened a new pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

holdenk opened a new pull request #34264:
URL: https://github.com/apache/spark/pull/34264


   
   ### What changes were proposed in this pull request?
   
   Add the ability to selectively disable watching or polling
   
   ### Why are the changes needed?
   
   Watching or polling for pod status on Kubernetes can place additional load on etcd, with a large number of executors and large number of jobs this can have negative impacts and executors register themselves with the driver under normal operations anyways.
   
   ### Does this PR introduce _any_ user-facing change?
   
   Two new config flags.
   
   
   ### How was this patch tested?
   
   New unit tests + manually tested a forked version of this on an internal cluster with both watching and polling disabled.


-- 
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 #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

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


   BTW, in general, I agree with your demands and requirements in this PR. The only concerns are
   - the better backward compatibility
   - the visibility of these configurations


-- 
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 #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

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


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/145072/
   


-- 
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 a change in pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #34264:
URL: https://github.com/apache/spark/pull/34264#discussion_r758938725



##########
File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
##########
@@ -395,6 +395,23 @@ private[spark] object Config extends Logging {
       .checkValue(interval => interval > 0, s"Logging interval must be a positive time value.")
       .createWithDefaultString("1s")
 
+  val KUBERNETES_EXECUTOR_ENABLE_API_POLLING =
+    ConfigBuilder("spark.kubernetes.executor.enableApiPolling")
+      .doc("If Spark should poll Kubernetes for executor pod status. " +
+        "You should leave this enabled unless your encountering performance issues with your etcd.")
+      .version("3.3.0")
+      .booleanConf
+      .createWithDefault(true)

Review comment:
       I feel the warning, `You should leave`, is insufficient. Can we make this `internal` conf?




-- 
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] holdenk commented on pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
holdenk commented on pull request #34264:
URL: https://github.com/apache/spark/pull/34264#issuecomment-965851610


   jenkins retest this please.


-- 
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] SparkQA commented on pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34264:
URL: https://github.com/apache/spark/pull/34264#issuecomment-941333042






-- 
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] SparkQA commented on pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34264:
URL: https://github.com/apache/spark/pull/34264#issuecomment-941333042


   **[Test build #144164 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144164/testReport)** for PR 34264 at commit [`65d7cc7`](https://github.com/apache/spark/commit/65d7cc7ffe7513c84f1a66e118cf23d747ad1805).


-- 
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] SparkQA removed a comment on pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #34264:
URL: https://github.com/apache/spark/pull/34264#issuecomment-941333042


   **[Test build #144164 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144164/testReport)** for PR 34264 at commit [`65d7cc7`](https://github.com/apache/spark/commit/65d7cc7ffe7513c84f1a66e118cf23d747ad1805).


-- 
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] github-actions[bot] commented on pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #34264:
URL: https://github.com/apache/spark/pull/34264#issuecomment-1065703183


   We're closing this PR because it hasn't been updated in a while. This isn't a judgement on the merit of the PR in any way. It's just a way of keeping the PR queue manageable.
   If you'd like to revive this PR, please reopen it and ask a committer to remove the Stale tag!


-- 
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 removed a comment on pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34264:
URL: https://github.com/apache/spark/pull/34264#issuecomment-941440370


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/144164/
   


-- 
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 removed a comment on pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34264:
URL: https://github.com/apache/spark/pull/34264#issuecomment-965882083






-- 
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 a change in pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #34264:
URL: https://github.com/apache/spark/pull/34264#discussion_r758938817



##########
File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
##########
@@ -395,6 +395,23 @@ private[spark] object Config extends Logging {
       .checkValue(interval => interval > 0, s"Logging interval must be a positive time value.")
       .createWithDefaultString("1s")
 
+  val KUBERNETES_EXECUTOR_ENABLE_API_POLLING =
+    ConfigBuilder("spark.kubernetes.executor.enableApiPolling")
+      .doc("If Spark should poll Kubernetes for executor pod status. " +
+        "You should leave this enabled unless your encountering performance issues with your etcd.")
+      .version("3.3.0")
+      .booleanConf
+      .createWithDefault(true)
+
+  val KUBERNETES_EXECUTOR_ENABLE_API_WATCHER =
+    ConfigBuilder("spark.kubernetes.executor.enableApiWatcher")
+      .doc("If Spark should create watchers for executor pod status. " +
+        "You should leave this enabled unless your encountering performance issues with your etcd.")
+      .version("3.3.0")
+      .booleanConf
+      .createWithDefault(true)

Review comment:
       Ditto. Can we make it as an `internal` conf?




-- 
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 removed a comment on pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34264:
URL: https://github.com/apache/spark/pull/34264#issuecomment-941440370






-- 
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 removed a comment on pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #34264:
URL: https://github.com/apache/spark/pull/34264#issuecomment-941562631


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48642/
   


-- 
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] SparkQA removed a comment on pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #34264:
URL: https://github.com/apache/spark/pull/34264#issuecomment-965861671


   **[Test build #145072 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/145072/testReport)** for PR 34264 at commit [`65d7cc7`](https://github.com/apache/spark/commit/65d7cc7ffe7513c84f1a66e118cf23d747ad1805).


-- 
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] SparkQA commented on pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34264:
URL: https://github.com/apache/spark/pull/34264#issuecomment-965881924


   **[Test build #145072 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/145072/testReport)** for PR 34264 at commit [`65d7cc7`](https://github.com/apache/spark/commit/65d7cc7ffe7513c84f1a66e118cf23d747ad1805).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


-- 
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 a change in pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #34264:
URL: https://github.com/apache/spark/pull/34264#discussion_r758934740



##########
File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
##########
@@ -395,6 +395,23 @@ private[spark] object Config extends Logging {
       .checkValue(interval => interval > 0, s"Logging interval must be a positive time value.")
       .createWithDefaultString("1s")
 
+  val KUBERNETES_EXECUTOR_ENABLE_API_POLLING =
+    ConfigBuilder("spark.kubernetes.executor.enableApiPolling")
+      .doc("If Spark should poll Kubernetes for executor pod status. " +
+        "You should leave this enabled unless your encountering performance issues with your etcd.")
+      .version("3.3.0")
+      .booleanConf
+      .createWithDefault(true)
+
+  val KUBERNETES_EXECUTOR_ENABLE_API_WATCHER =
+    ConfigBuilder("spark.kubernetes.executor.enableApiWatcher")
+      .doc("If Spark should create watchers for executor pod status. " +
+        "You should leave this enabled unless your encountering performance issues with your etcd.")
+      .version("3.3.0")
+      .booleanConf
+      .createWithDefault(true)
+
+

Review comment:
       nit. extra empty line.




-- 
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 #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

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


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/144164/
   


-- 
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 #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

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


   Sorry for being late, @holdenk . I'll take a look at this now.


-- 
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 a change in pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #34264:
URL: https://github.com/apache/spark/pull/34264#discussion_r758971094



##########
File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSource.scala
##########
@@ -22,24 +22,30 @@ import io.fabric8.kubernetes.api.model.Pod
 import io.fabric8.kubernetes.client.{KubernetesClient, Watcher, WatcherException}
 import io.fabric8.kubernetes.client.Watcher.Action
 
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.k8s.Config.KUBERNETES_EXECUTOR_ENABLE_API_WATCHER
 import org.apache.spark.deploy.k8s.Constants._
 import org.apache.spark.internal.Logging
 import org.apache.spark.util.Utils
 
 private[spark] class ExecutorPodsWatchSnapshotSource(
     snapshotsStore: ExecutorPodsSnapshotsStore,
-    kubernetesClient: KubernetesClient) extends Logging {
+    kubernetesClient: KubernetesClient,
+    conf: SparkConf) extends Logging {

Review comment:
       For this one, I made a PR to provide a better backward compatibility and to help the downstreams' ExternalClusterManager. Since these classes are unchanged since 2.4.0, I believe we can declare it `stable developer API` and maintain it more carefully.
   - https://github.com/apache/spark/pull/34751




-- 
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 #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

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


   Could you rebase this PR please, @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] SparkQA commented on pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34264:
URL: https://github.com/apache/spark/pull/34264#issuecomment-941522026


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48642/
   


-- 
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] SparkQA commented on pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34264:
URL: https://github.com/apache/spark/pull/34264#issuecomment-965861671


   **[Test build #145072 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/145072/testReport)** for PR 34264 at commit [`65d7cc7`](https://github.com/apache/spark/commit/65d7cc7ffe7513c84f1a66e118cf23d747ad1805).


-- 
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] SparkQA commented on pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34264:
URL: https://github.com/apache/spark/pull/34264#issuecomment-941358113


   **[Test build #144164 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144164/testReport)** for PR 34264 at commit [`65d7cc7`](https://github.com/apache/spark/commit/65d7cc7ffe7513c84f1a66e118cf23d747ad1805).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


-- 
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] github-actions[bot] closed pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
github-actions[bot] closed pull request #34264:
URL: https://github.com/apache/spark/pull/34264


   


-- 
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 #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

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


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/49541/
   


-- 
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] SparkQA commented on pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34264:
URL: https://github.com/apache/spark/pull/34264#issuecomment-965893108


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/49541/
   


-- 
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] SparkQA removed a comment on pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #34264:
URL: https://github.com/apache/spark/pull/34264#issuecomment-941333042


   **[Test build #144164 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/144164/testReport)** for PR 34264 at commit [`65d7cc7`](https://github.com/apache/spark/commit/65d7cc7ffe7513c84f1a66e118cf23d747ad1805).


-- 
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 #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

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


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/48642/
   


-- 
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] SparkQA commented on pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34264:
URL: https://github.com/apache/spark/pull/34264#issuecomment-965911189


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/49541/
   


-- 
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] holdenk commented on pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
holdenk commented on pull request #34264:
URL: https://github.com/apache/spark/pull/34264#issuecomment-972031938


   Hey @dongjoon-hyun do you have any cycles for this? The failures are unrelated (I have a cleanup PR for the K8s int test failures - https://github.com/apache/spark/pull/34636 )


-- 
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 a change in pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #34264:
URL: https://github.com/apache/spark/pull/34264#discussion_r758937355



##########
File path: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsWatchSnapshotSource.scala
##########
@@ -22,24 +22,30 @@ import io.fabric8.kubernetes.api.model.Pod
 import io.fabric8.kubernetes.client.{KubernetesClient, Watcher, WatcherException}
 import io.fabric8.kubernetes.client.Watcher.Action
 
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.k8s.Config.KUBERNETES_EXECUTOR_ENABLE_API_WATCHER
 import org.apache.spark.deploy.k8s.Constants._
 import org.apache.spark.internal.Logging
 import org.apache.spark.util.Utils
 
 private[spark] class ExecutorPodsWatchSnapshotSource(
     snapshotsStore: ExecutorPodsSnapshotsStore,
-    kubernetesClient: KubernetesClient) extends Logging {
+    kubernetesClient: KubernetesClient,
+    conf: SparkConf) extends Logging {

Review comment:
       Although this is a `private` class, Spark supports `ExternalClusterManager` and some implementations depend on this class. Shall we improve this PR in a way to keep a backward compatibility? I guess we can keep old constructer and add new constructor additionally.
   
   cc @shrutig since this is a breaking change in your context.




-- 
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 a change in pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #34264:
URL: https://github.com/apache/spark/pull/34264#discussion_r758938108



##########
File path: resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSourceSuite.scala
##########
@@ -89,13 +90,27 @@ class ExecutorPodsPollingSnapshotSourceSuite extends SparkFunSuite with BeforeAn
     verify(eventQueue).replaceSnapshot(Seq(exec1, exec2))
   }
 
+  test("If polling is disabled we don't call pods() on the client") {

Review comment:
       nit. If you don't mind, could you add a test prefix, `SPARK-36462:`?




-- 
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] SparkQA commented on pull request #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #34264:
URL: https://github.com/apache/spark/pull/34264#issuecomment-941429522


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/48642/
   


-- 
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 #34264: [SPARK-36462][K8S] Add the ability to selectively disable watching or polling

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






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