This is an automated email from the ASF dual-hosted git repository.

gurwls223 pushed a commit to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.0 by this push:
     new 6d7ae4a  [SPARK-32160][CORE][PYSPARK][3.0][FOLLOWUP] Change the config 
name to switch allow/disallow SparkContext in executors
6d7ae4a is described below

commit 6d7ae4a557b89d87a639c58bcd94c96763b8d5a7
Author: Takuya UESHIN <[email protected]>
AuthorDate: Tue Aug 4 12:43:01 2020 +0900

    [SPARK-32160][CORE][PYSPARK][3.0][FOLLOWUP] Change the config name to 
switch allow/disallow SparkContext in executors
    
    ### What changes were proposed in this pull request?
    
    This is a follow-up of #29294.
    This PR changes the config name to switch allow/disallow `SparkContext` in 
executors as per the comment 
https://github.com/apache/spark/pull/29278#pullrequestreview-460256338.
    
    ### Why are the changes needed?
    
    The config name `spark.executor.allowSparkContext` is more reasonable.
    
    ### Does this PR introduce _any_ user-facing change?
    
    Yes, the config name is changed.
    
    ### How was this patch tested?
    
    Updated tests.
    
    Closes #29341 from ueshin/issues/SPARK-32160/3.0/change_config_name.
    
    Authored-by: Takuya UESHIN <[email protected]>
    Signed-off-by: HyukjinKwon <[email protected]>
---
 core/src/main/scala/org/apache/spark/SparkContext.scala               | 2 +-
 core/src/main/scala/org/apache/spark/internal/config/package.scala    | 4 ++--
 core/src/test/scala/org/apache/spark/SparkContextSuite.scala          | 2 +-
 python/pyspark/context.py                                             | 2 +-
 python/pyspark/tests/test_context.py                                  | 2 +-
 sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala       | 4 ++--
 .../test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala    | 4 ++--
 7 files changed, 10 insertions(+), 10 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala 
b/core/src/main/scala/org/apache/spark/SparkContext.scala
index c448eee..65c08cf 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 38eb90c..0ff6b73 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 5533f42..ce437a5 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 3c447a1..72e8e84 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -120,7 +120,7 @@ class SparkContext(object):
         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 9468b25..29b49fd 100644
--- a/python/pyspark/tests/test_context.py
+++ b/python/pyspark/tests/test_context.py
@@ -271,7 +271,7 @@ class ContextTests(unittest.TestCase):
         # 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 6c809c8..fc94127 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 6983cda..159d2c0 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()


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to