[
https://issues.apache.org/jira/browse/SPARK-42596?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17693837#comment-17693837
]
John Zhuge commented on SPARK-42596:
Looks like a regression from SPARK-41188 where it removed the code that sets
the default OMP_NUM_THREADS from PythonRunner.
Its PR assumes the code can be moved to SparkContext, unfortunately
`SparkContext#executorEnvs` is only used by StandaloneSchedulerBackend for
Spark's standalone cluster manager, thus the PR broke YARN as shown in the test
case above, probably Mesos as well but I don't have a way to test.
> [YARN] OMP_NUM_THREADS not set to number of executor cores by default
> -
>
> Key: SPARK-42596
> URL: https://issues.apache.org/jira/browse/SPARK-42596
> Project: Spark
> Issue Type: Bug
> Components: PySpark, YARN
>Affects Versions: 3.3.2
>Reporter: John Zhuge
>Priority: Major
>
> Run this PySpark script with `spark.executor.cores=1`
> {code:python}
> import os
> from pyspark.sql import SparkSession
> from pyspark.sql.functions import udf
> spark = SparkSession.builder.getOrCreate()
> var_name = 'OMP_NUM_THREADS'
> def get_env_var():
> return os.getenv(var_name)
> udf_get_env_var = udf(get_env_var)
> spark.range(1).toDF("id").withColumn(f"env_{var_name}",
> udf_get_env_var()).show(truncate=False)
> {code}
> Output with release `3.3.2`:
> {noformat}
> +---+---+
> |id |env_OMP_NUM_THREADS|
> +---+---+
> |0 |null |
> +---+---+
> {noformat}
> Output with release `3.3.0`:
> {noformat}
> +---+---+
> |id |env_OMP_NUM_THREADS|
> +---+---+
> |0 |1 |
> +---+---+
> {noformat}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org