diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c448eee5fca23..65c08cff2ba11 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -82,7 +82,7 @@ class SparkContext(config: SparkConf) extends Logging { // The call site where this SparkContext was constructed. private val creationSite: CallSite = Utils.getCallSite() - if (!config.get(ALLOW_SPARK_CONTEXT_IN_EXECUTORS)) { + if (!config.get(EXECUTOR_ALLOW_SPARK_CONTEXT)) { // In order to prevent SparkContext from being created in executors. SparkContext.assertOnDriver() } diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 38eb90c57ef68..0ff6b73d20dff 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1814,8 +1814,8 @@ package object config { .bytesConf(ByteUnit.BYTE) .createOptional - private[spark] val ALLOW_SPARK_CONTEXT_IN_EXECUTORS = - ConfigBuilder("spark.driver.allowSparkContextInExecutors") + private[spark] val EXECUTOR_ALLOW_SPARK_CONTEXT = + ConfigBuilder("spark.executor.allowSparkContext") .doc("If set to true, SparkContext can be created in executors.") .version("3.0.1") .booleanConf diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 5533f42859bd1..ce437a5e42b22 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -957,7 +957,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu val error = intercept[SparkException] { sc.range(0, 1).foreach { _ => new SparkContext(new SparkConf().setAppName("test").setMaster("local") - .set(ALLOW_SPARK_CONTEXT_IN_EXECUTORS, false)) + .set(EXECUTOR_ALLOW_SPARK_CONTEXT, false)) } }.getMessage() diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 3c447a10b7058..72e8e84cba757 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -120,7 +120,7 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, ValueError:... """ if (conf is not None and - conf.get("spark.driver.allowSparkContextInExecutors", "true").lower() != "true"): + conf.get("spark.executor.allowSparkContext", "true").lower() != "true"): # In order to prevent SparkContext from being created in executors. SparkContext._assert_on_driver() diff --git a/python/pyspark/tests/test_context.py b/python/pyspark/tests/test_context.py index 9468b2511ce03..29b49fd4417e1 100644 --- a/python/pyspark/tests/test_context.py +++ b/python/pyspark/tests/test_context.py @@ -271,7 +271,7 @@ def test_disallow_to_create_spark_context_in_executors(self): # SPARK-32160: SparkContext should not created in executors if the config is set. def create_spark_context(): - conf = SparkConf().set("spark.driver.allowSparkContextInExecutors", "false") + conf = SparkConf().set("spark.executor.allowSparkContext", "false") with SparkContext(conf=conf): pass diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 6c809c8592522..fc941270e1b64 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -29,7 +29,7 @@ import org.apache.spark.{SPARK_VERSION, SparkConf, SparkContext, TaskContext} import org.apache.spark.annotation.{DeveloperApi, Experimental, Stable, Unstable} import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.ALLOW_SPARK_CONTEXT_IN_EXECUTORS +import org.apache.spark.internal.config.EXECUTOR_ALLOW_SPARK_CONTEXT import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd} import org.apache.spark.sql.catalog.Catalog @@ -904,7 +904,7 @@ object SparkSession extends Logging { val sparkConf = new SparkConf() options.foreach { case (k, v) => sparkConf.set(k, v) } - if (!sparkConf.get(ALLOW_SPARK_CONTEXT_IN_EXECUTORS)) { + if (!sparkConf.get(EXECUTOR_ALLOW_SPARK_CONTEXT)) { assertOnDriver() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala index 6983cda5a35cc..159d2c02188be 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import org.scalatest.BeforeAndAfterEach import org.apache.spark.{SparkConf, SparkContext, SparkException, SparkFunSuite} -import org.apache.spark.internal.config.ALLOW_SPARK_CONTEXT_IN_EXECUTORS +import org.apache.spark.internal.config.EXECUTOR_ALLOW_SPARK_CONTEXT import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf._ @@ -248,7 +248,7 @@ class SparkSessionBuilderSuite extends SparkFunSuite with BeforeAndAfterEach { val error = intercept[SparkException] { session.range(1).foreach { v => SparkSession.builder.master("local") - .config(ALLOW_SPARK_CONTEXT_IN_EXECUTORS.key, false).getOrCreate() + .config(EXECUTOR_ALLOW_SPARK_CONTEXT.key, false).getOrCreate() () } }.getMessage()