From 9c5d7c4df5144addd0484a6b2436354441aa1c17 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Fri, 27 Oct 2023 15:25:16 +0900 Subject: [PATCH] [SPARK-45670][CORE][3.4] SparkSubmit does not support `--total-executor-cores` when deploying on K8s This is the cherry-pick of https://github.com/apache/spark/pull/43536 for branch-3.4 ### What changes were proposed in this pull request? Remove Kubernetes from the support list of `--total-executor-cores` in SparkSubmit ### Why are the changes needed? `--total-executor-cores` does not take effect in Spark on K8s, [the comments from original PR](https://github.com/apache/spark/pull/19717#discussion_r154568773) also proves that ### Does this PR introduce _any_ user-facing change? The output of `spark-submit --help` changed ```patch ... - Spark standalone, Mesos and Kubernetes only: + Spark standalone and Mesos only: --total-executor-cores NUM Total cores for all executors. ... ``` ### How was this patch tested? Pass GA and review. ### Was this patch authored or co-authored using generative AI tooling? No Closes #43549 from pan3793/SPARK-45670-3.4. Authored-by: Cheng Pan Signed-off-by: Hyukjin Kwon --- core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 2 +- .../scala/org/apache/spark/deploy/SparkSubmitArguments.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 506dd0d916a1f..8b4ef1dee8acb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -679,7 +679,7 @@ private[spark] class SparkSubmit extends Logging { confKey = EXECUTOR_CORES.key), OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN | KUBERNETES, ALL_DEPLOY_MODES, confKey = EXECUTOR_MEMORY.key), - OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS | KUBERNETES, ALL_DEPLOY_MODES, + OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, ALL_DEPLOY_MODES, confKey = CORES_MAX.key), OptionAssigner(args.files, LOCAL | STANDALONE | MESOS | KUBERNETES, ALL_DEPLOY_MODES, confKey = FILES.key), diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index a3fe5153bee9f..93dd25db0937b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -567,7 +567,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | --kill SUBMISSION_ID If given, kills the driver specified. | --status SUBMISSION_ID If given, requests the status of the driver specified. | - | Spark standalone, Mesos and Kubernetes only: + | Spark standalone and Mesos only: | --total-executor-cores NUM Total cores for all executors. | | Spark standalone, YARN and Kubernetes only: