From 801e4d097f45b269a9c6b25723d925f3e24ba498 Mon Sep 17 00:00:00 2001
From: Sean Owen <sowen@cloudera.com>
Date: Tue, 9 Aug 2016 09:38:12 +0100
Subject: [PATCH] [SPARK-16606][CORE] Misleading warning for
 SparkContext.getOrCreate "WARN SparkContext: Use an existing SparkContext,
 some configuration may not take effect."

## What changes were proposed in this pull request?

SparkContext.getOrCreate shouldn't warn about ignored config if

- it wasn't ignored because a new context is created with it or
- no config was actually provided

## How was this patch tested?

Jenkins + existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #14533 from srowen/SPARK-16606.
---
 .../main/scala/org/apache/spark/SparkContext.scala | 14 ++++++++++----
 1 file changed, 10 insertions(+), 4 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index fc7ea51ab0..4f3bb1c877 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -2262,9 +2262,10 @@ object SparkContext extends Logging {
     SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized {
       if (activeContext.get() == null) {
         setActiveContext(new SparkContext(config), allowMultipleContexts = false)
-      }
-      if (config.getAll.nonEmpty) {
-        logWarning("Use an existing SparkContext, some configuration may not take effect.")
+      } else {
+        if (config.getAll.nonEmpty) {
+          logWarning("Using an existing SparkContext; some configuration may not take effect.")
+        }
       }
       activeContext.get()
     }
@@ -2281,7 +2282,12 @@ object SparkContext extends Logging {
    * even if multiple contexts are allowed.
    */
   def getOrCreate(): SparkContext = {
-    getOrCreate(new SparkConf())
+    SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized {
+      if (activeContext.get() == null) {
+        setActiveContext(new SparkContext(), allowMultipleContexts = false)
+      }
+      activeContext.get()
+    }
   }
 
   /**
-- 
GitLab