You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by jerryshao <gi...@git.apache.org> on 2018/01/26 09:46:35 UTC

[GitHub] spark pull request #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkS...

GitHub user jerryshao opened a pull request:

    https://github.com/apache/spark/pull/20404

    [SPARK-23228][PYSPARK] Add Python Created jsparkSession to JVM's defaultSession

    ## What changes were proposed in this pull request?
    
    In the current PySpark code, Python created `jsparkSession` doesn't add to JVM's defaultSession, this `SparkSession` object cannot be fetched from Java side, so the below scala code will be failed when loaded in PySpark application.
    
    ```scala
    class TestSparkSession extends SparkListener with Logging {
      override def onOtherEvent(event: SparkListenerEvent): Unit = {
        event match {
          case CreateTableEvent(db, table) =>
            val session = SparkSession.getActiveSession.orElse(SparkSession.getDefaultSession)
            assert(session.isDefined)
            val tableInfo = session.get.sharedState.externalCatalog.getTable(db, table)
            logInfo(s"Table info ${tableInfo}")
    
          case e =>
            logInfo(s"event $e")
    
        }
      }
    }
    ```
    
    So here propose to add fresh create `jsparkSession` to `defaultSession`.
    
    ## How was this patch tested?
    
    Manual verification.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/jerryshao/apache-spark SPARK-23228

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/20404.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #20404
    
----
commit d9189ad1763d4ef867027f4ade2a332d589fe698
Author: jerryshao <ss...@...>
Date:   2018-01-26T09:41:28Z

    Add Python Create jsparkSession to JVM's defaultSession

----


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Thanks Felix. I would incline to not fix the case mentioned by Felix. What's your opinion @HyukjinKwon  @ueshin ?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    **[Test build #86766 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86766/testReport)** for PR 20404 at commit [`cc4b851`](https://github.com/apache/spark/commit/cc4b8510c1445fb742c0d750958d352adfa84902).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/329/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkS...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20404#discussion_r164109079
  
    --- Diff: python/pyspark/sql/session.py ---
    @@ -225,6 +225,7 @@ def __init__(self, sparkContext, jsparkSession=None):
             if SparkSession._instantiatedSession is None \
                     or SparkSession._instantiatedSession._sc._jsc is None:
                 SparkSession._instantiatedSession = self
    +            self._jvm.org.apache.spark.sql.SparkSession.setDefaultSession(self._jsparkSession)
    --- End diff --
    
    Actually, It seems not because we don't call this code path. Stop and start logic is convoluted in PySpark in my humble opinion. Setting the default one fixes an actual issue and seems we are okay with it, at least.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    I think I made a duplicated effort .. thanks for taking this in.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    I am fine with not fixing the case here for now if we go for it only with master branch. Maybe, we can try to fix the pre existing issue too seperately by syncrhoziing everything if anyone feels strongly against within 2.4.0 timeline. To be honest, the case sounds rare and the fix could be a little bit overkill .. 



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    **[Test build #86752 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86752/testReport)** for PR 20404 at commit [`eec4386`](https://github.com/apache/spark/commit/eec438619607008ef50e85399a90f6b0ce1e70f1).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    **[Test build #86759 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86759/testReport)** for PR 20404 at commit [`ec94c05`](https://github.com/apache/spark/commit/ec94c054a8e0abf2d13434556d0434b4690590d3).
     * This patch **fails Python style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    **[Test build #86765 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86765/testReport)** for PR 20404 at commit [`e5f4b58`](https://github.com/apache/spark/commit/e5f4b580d3ecf69f30638ca1bc36073e132c489e).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/332/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    @felixcheung what is your opinion on this, do we really need to handle this case?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    **[Test build #86756 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86756/testReport)** for PR 20404 at commit [`dd1c991`](https://github.com/apache/spark/commit/dd1c9912101389983011b47292da08eff0f43cdc).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkS...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20404#discussion_r164350646
  
    --- Diff: python/pyspark/sql/session.py ---
    @@ -760,6 +764,7 @@ def stop(self):
             """Stop the underlying :class:`SparkContext`.
             """
             self._sc.stop()
    +        self._jvm.SparkSession.clearDefaultSession()
    --- End diff --
    
    I'm also working on the failure, already figured out why.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Jenkins, retest this please.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkS...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20404#discussion_r164109814
  
    --- Diff: python/pyspark/sql/session.py ---
    @@ -225,6 +225,7 @@ def __init__(self, sparkContext, jsparkSession=None):
             if SparkSession._instantiatedSession is None \
                     or SparkSession._instantiatedSession._sc._jsc is None:
                 SparkSession._instantiatedSession = self
    +            self._jvm.org.apache.spark.sql.SparkSession.setDefaultSession(self._jsparkSession)
    --- End diff --
    
    Ohh, I see. My misunderstanding. let me figure out a way to clear this object.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/86760/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkS...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20404#discussion_r164108216
  
    --- Diff: python/pyspark/sql/session.py ---
    @@ -225,6 +225,7 @@ def __init__(self, sparkContext, jsparkSession=None):
             if SparkSession._instantiatedSession is None \
                     or SparkSession._instantiatedSession._sc._jsc is None:
                 SparkSession._instantiatedSession = self
    +            self._jvm.org.apache.spark.sql.SparkSession.setDefaultSession(self._jsparkSession)
    --- End diff --
    
    >Btw, shall we clear it when stopping PySpark SparkSession?
    
    JVM SparkSession will clear it when application is stopped (https://github.com/apache/spark/blob/3e252514741447004f3c18ddd77c617b4e37cfaa/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala#L961).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/328/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    **[Test build #86700 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86700/testReport)** for PR 20404 at commit [`d9189ad`](https://github.com/apache/spark/commit/d9189ad1763d4ef867027f4ade2a332d589fe698).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    **[Test build #86766 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86766/testReport)** for PR 20404 at commit [`cc4b851`](https://github.com/apache/spark/commit/cc4b8510c1445fb742c0d750958d352adfa84902).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Thanks all for your comments. I think @felixcheung 's case really makes thing complex, I'm not sure if user will use it in such way. I will try to address it. Appreciate your comments!


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    **[Test build #86758 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86758/testReport)** for PR 20404 at commit [`dd1c991`](https://github.com/apache/spark/commit/dd1c9912101389983011b47292da08eff0f43cdc).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    How about something like this?
    
    ```diff
    --- a/python/pyspark/sql/session.py
    +++ b/python/pyspark/sql/session.py
    @@ -213,10 +213,10 @@ class SparkSession(object):
             self._jsc = self._sc._jsc
             self._jvm = self._sc._jvm
             if jsparkSession is None:
    -            if self._jvm.SparkSession.getDefaultSession().isDefined() \
    -                    and not self._jvm.SparkSession.getDefaultSession().get() \
    -                        .sparkContext().isStopped():
    -                jsparkSession = self._jvm.SparkSession.getDefaultSession().get()
    +            default_session = self._jvm.SparkSession.getDefaultSession()
    +            if default_session.isDefined() \
    +                    and not default_session.get().sparkContext().isStopped():
    +                jsparkSession = default_session
                 else:
                     jsparkSession = self._jvm.SparkSession(self._jsc.sc())
             self._jsparkSession = jsparkSession
    @@ -230,7 +230,7 @@ class SparkSession(object):
             if SparkSession._instantiatedSession is None \
                     or SparkSession._instantiatedSession._sc._jsc is None:
                 SparkSession._instantiatedSession = self
    -            self._jvm.SparkSession.setDefaultSession(self._jsparkSession)
    +            self._jvm.SparkSession.setDefaultSessionIfUnset(self._jsparkSession)
    
         def _repr_html_(self):
             return """
    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 734573ba31f..99747ef88bc 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
    @@ -1031,6 +1031,18 @@ object SparkSession {
       // Private methods from now on
       ////////////////////////////////////////////////////////////////////////////////////////
    
    +  /**
    +   * This if for Python blabla
    +   */
    +  private[sql] def setDefaultSessionIfUnset(session: SparkSession): Unit = {
    +    // Global synchronization blabla
    +    SparkSession.synchronized {
    +      if (defaultSession.get() eq null) {
    +        defaultSession.set(session)
    +      }
    +    }
    +  }
    +
       /** The active SparkSession for the current thread. */
       private val activeThreadSession = new InheritableThreadLocal[SparkSession]
    ```



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    How about you @ueshin?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/86756/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    **[Test build #86762 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86762/testReport)** for PR 20404 at commit [`1ed62ef`](https://github.com/apache/spark/commit/1ed62ef1dbc2c46f2d21eb6147418a86bd59c55c).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/321/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    **[Test build #86752 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86752/testReport)** for PR 20404 at commit [`eec4386`](https://github.com/apache/spark/commit/eec438619607008ef50e85399a90f6b0ce1e70f1).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Yup, sorry for rushing the suggestion. Wanted to show an idea. BTW, if we should replace it if it's already stopped, I assume we need the synchronization with `defaultSession` anyway if I didn't misunderstand. Did I maybe miss something?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    @zjffdu @HyukjinKwon please help to review. Thanks!


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/280/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/86700/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/86766/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by ueshin <gi...@git.apache.org>.
Github user ueshin commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    @HyukjinKwon I'm afraid that's not enough because we need to synchronize at all the places we touch the `defaultSession`, otherwise another thread can set the value between `defaultSession.get() eq null` and `defaultSession.set(session)`.
    
    Fortunately `defaultSession` is an `AtomicReference`, so maybe we can use `defaultSession.compareAndSet(null, session)` without the synchronization in the method `setDefaultSessionIfUnset()` if we only need to set the default session is `null`.
    
    I'm not sure whether we want to replace the default session which is already stopped, but in that case, we may also have to send the original default session to compare.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Thanks all for your review, greatly appreciated.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkS...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20404#discussion_r164103440
  
    --- Diff: python/pyspark/sql/session.py ---
    @@ -225,6 +225,7 @@ def __init__(self, sparkContext, jsparkSession=None):
             if SparkSession._instantiatedSession is None \
                     or SparkSession._instantiatedSession._sc._jsc is None:
                 SparkSession._instantiatedSession = self
    +            self._jvm.org.apache.spark.sql.SparkSession.setDefaultSession(self._jsparkSession)
    --- End diff --
    
    `setActiveSession` or `setDefaultSession`? Which one is more proper to set here?
    
    Btw, shall we clear it when stopping PySpark SparkSession?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/327/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkS...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/20404


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by ueshin <gi...@git.apache.org>.
Github user ueshin commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    @felixcheung I see, in that case, we should revert the last commit (cc4b8510c1445fb742c0d750958d352adfa84902) to check the default session is updated or not?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Thanks @HyukjinKwon for your help.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    LGTM


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/337/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkS...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20404#discussion_r164253931
  
    --- Diff: python/pyspark/sql/session.py ---
    @@ -225,6 +225,7 @@ def __init__(self, sparkContext, jsparkSession=None):
             if SparkSession._instantiatedSession is None \
                     or SparkSession._instantiatedSession._sc._jsc is None:
                 SparkSession._instantiatedSession = self
    +            self._jvm.org.apache.spark.sql.SparkSession.setDefaultSession(self._jsparkSession)
    --- End diff --
    
    Yeah, I think we can clear it in `def stop`.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/330/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/86752/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkS...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20404#discussion_r164266681
  
    --- Diff: python/pyspark/sql/session.py ---
    @@ -225,6 +225,7 @@ def __init__(self, sparkContext, jsparkSession=None):
             if SparkSession._instantiatedSession is None \
                     or SparkSession._instantiatedSession._sc._jsc is None:
                 SparkSession._instantiatedSession = self
    +            self._jvm.org.apache.spark.sql.SparkSession.setDefaultSession(self._jsparkSession)
    --- End diff --
    
    I think this is somewhat conflicted ... If I can SparkSession.getOrCreate in python and JVM already has a defaultSession, what should happen?
    I suppose one approach would be to keep python independent.. but with this change it would overwrite the defaultSession that might be valid?



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkS...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20404#discussion_r164364573
  
    --- Diff: python/pyspark/sql/session.py ---
    @@ -225,6 +230,9 @@ def __init__(self, sparkContext, jsparkSession=None):
             if SparkSession._instantiatedSession is None \
                     or SparkSession._instantiatedSession._sc._jsc is None:
                 SparkSession._instantiatedSession = self
    +            if self._jvm.SparkSession.getDefaultSession().isEmpty() \
    +                    or not jsparkSession.equals(self._jvm.SparkSession.getDefaultSession().get()):
    +                self._jvm.SparkSession.setDefaultSession(self._jsparkSession)
    --- End diff --
    
    @felixcheung has concern about simply overwriting the default session.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    I think the same issue also existed in Scala `SparkSession` code, because `setDefaultSession` doesn't hold a lock which holds by `getOrCreate` (SparkSession).
    
    For example:
    
    ```scala
    
    val globalSession = SparkSession.build.getOrCreate()
    val session1 = globalSession.newSession()
    
    globalSession.sparkContext.stop()
    
    // Thread1
    val localSession = SparkSession.build.getOrCreate()
    
    // Thread2 set default session during Thread1's getOrCreate
    SparkSession.setDefaultSession(session1)
    
    // After Thread1's getOrCreate is finished, it will still overwrite the default session to "localSession", not the one (session1) user explicitly set 
    ```
    
    I think this is the similar scenario as mentioned for PySpark.
    
    
    
    
    



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by ueshin <gi...@git.apache.org>.
Github user ueshin commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    I completely agree with @HyukjinKwon.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    **[Test build #86700 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86700/testReport)** for PR 20404 at commit [`d9189ad`](https://github.com/apache/spark/commit/d9189ad1763d4ef867027f4ade2a332d589fe698).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkS...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20404#discussion_r164105856
  
    --- Diff: python/pyspark/sql/session.py ---
    @@ -225,6 +225,7 @@ def __init__(self, sparkContext, jsparkSession=None):
             if SparkSession._instantiatedSession is None \
                     or SparkSession._instantiatedSession._sc._jsc is None:
                 SparkSession._instantiatedSession = self
    +            self._jvm.org.apache.spark.sql.SparkSession.setDefaultSession(self._jsparkSession)
    --- End diff --
    
    By looking at Scala code, seems Scala `getOrCreate` will set to `defaultSession` I was thinking it is more proper to set to `defaultSession`. Does PySpark support multiple sessions?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkS...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20404#discussion_r164349606
  
    --- Diff: python/pyspark/sql/session.py ---
    @@ -760,6 +764,7 @@ def stop(self):
             """Stop the underlying :class:`SparkContext`.
             """
             self._sc.stop()
    +        self._jvm.SparkSession.clearDefaultSession()
    --- End diff --
    
    hmm.. If we didn't set it in L231, perhaps we shouldn't clear it?
    Or if we are picking up the JVM one in L217, we shouldn't clear it either?
    
    WDYT?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    **[Test build #86760 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86760/testReport)** for PR 20404 at commit [`d9f77ea`](https://github.com/apache/spark/commit/d9f77eaadecedc115dd2dc64843761a3bb08d46e).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    **[Test build #86765 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86765/testReport)** for PR 20404 at commit [`e5f4b58`](https://github.com/apache/spark/commit/e5f4b580d3ecf69f30638ca1bc36073e132c489e).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    **[Test build #86758 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86758/testReport)** for PR 20404 at commit [`dd1c991`](https://github.com/apache/spark/commit/dd1c9912101389983011b47292da08eff0f43cdc).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/86758/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkS...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20404#discussion_r164364286
  
    --- Diff: python/pyspark/sql/session.py ---
    @@ -213,7 +213,12 @@ def __init__(self, sparkContext, jsparkSession=None):
             self._jsc = self._sc._jsc
             self._jvm = self._sc._jvm
             if jsparkSession is None:
    -            jsparkSession = self._jvm.SparkSession(self._jsc.sc())
    +            if self._jvm.SparkSession.getDefaultSession().isDefined() \
    +                    and not self._jvm.SparkSession.getDefaultSession().get() \
    +                        .sparkContext().isStopped():
    --- End diff --
    
    I think so. I will do it.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkS...

Posted by ueshin <gi...@git.apache.org>.
Github user ueshin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20404#discussion_r164362178
  
    --- Diff: python/pyspark/sql/session.py ---
    @@ -225,6 +230,9 @@ def __init__(self, sparkContext, jsparkSession=None):
             if SparkSession._instantiatedSession is None \
                     or SparkSession._instantiatedSession._sc._jsc is None:
                 SparkSession._instantiatedSession = self
    +            if self._jvm.SparkSession.getDefaultSession().isEmpty() \
    +                    or not jsparkSession.equals(self._jvm.SparkSession.getDefaultSession().get()):
    +                self._jvm.SparkSession.setDefaultSession(self._jsparkSession)
    --- End diff --
    
    Maybe we can simply overwrite the default session.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkS...

Posted by ueshin <gi...@git.apache.org>.
Github user ueshin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20404#discussion_r164372440
  
    --- Diff: python/pyspark/sql/session.py ---
    @@ -225,6 +230,9 @@ def __init__(self, sparkContext, jsparkSession=None):
             if SparkSession._instantiatedSession is None \
                     or SparkSession._instantiatedSession._sc._jsc is None:
                 SparkSession._instantiatedSession = self
    +            if self._jvm.SparkSession.getDefaultSession().isEmpty() \
    +                    or not jsparkSession.equals(self._jvm.SparkSession.getDefaultSession().get()):
    +                self._jvm.SparkSession.setDefaultSession(self._jsparkSession)
    --- End diff --
    
    I might miss something, but I guess @felixcheung's concern was fixed by checking if the default session is defined and not stopped so we can put the valid session or the same session from JVM without checking anymore.
    But I'm okay to leave it as it is as well.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkS...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20404#discussion_r164122074
  
    --- Diff: python/pyspark/sql/session.py ---
    @@ -225,6 +225,7 @@ def __init__(self, sparkContext, jsparkSession=None):
             if SparkSession._instantiatedSession is None \
                     or SparkSession._instantiatedSession._sc._jsc is None:
                 SparkSession._instantiatedSession = self
    +            self._jvm.org.apache.spark.sql.SparkSession.setDefaultSession(self._jsparkSession)
    --- End diff --
    
    Simplest way I can think of is just add it in `def stop`:
    
    ```
    self._jvm.org.apache.spark.sql.SparkSession.clearDefaultSession()
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkS...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20404#discussion_r164328055
  
    --- Diff: python/pyspark/sql/session.py ---
    @@ -225,6 +225,7 @@ def __init__(self, sparkContext, jsparkSession=None):
             if SparkSession._instantiatedSession is None \
                     or SparkSession._instantiatedSession._sc._jsc is None:
                 SparkSession._instantiatedSession = self
    +            self._jvm.org.apache.spark.sql.SparkSession.setDefaultSession(self._jsparkSession)
    --- End diff --
    
    Thanks @HyukjinKwon , let me update the code.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkS...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20404#discussion_r164350415
  
    --- Diff: python/pyspark/sql/session.py ---
    @@ -760,6 +764,7 @@ def stop(self):
             """Stop the underlying :class:`SparkContext`.
             """
             self._sc.stop()
    +        self._jvm.SparkSession.clearDefaultSession()
    --- End diff --
    
    BTW, let me make a PR to your branch @jerryshao to deal with the failure soon. I was looking into this out of my curiosity.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkS...

Posted by ueshin <gi...@git.apache.org>.
Github user ueshin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20404#discussion_r164361700
  
    --- Diff: python/pyspark/sql/session.py ---
    @@ -213,7 +213,12 @@ def __init__(self, sparkContext, jsparkSession=None):
             self._jsc = self._sc._jsc
             self._jvm = self._sc._jvm
             if jsparkSession is None:
    -            jsparkSession = self._jvm.SparkSession(self._jsc.sc())
    +            if self._jvm.SparkSession.getDefaultSession().isDefined() \
    +                    and not self._jvm.SparkSession.getDefaultSession().get() \
    +                        .sparkContext().isStopped():
    --- End diff --
    
    I guess this change at 4ba3aa2af1b7bbc69575c14fffed18d5f1f90d53 is enough to fix the previous test failure (`ERROR: test_sparksession_with_stopped_sparkcontext (pyspark.sql.tests.SQLTests2)`) and we can revert moving `self._jvm.SparkSession.clearDefaultSession()` to `SparkContext.stop()` at 
    0319fa5c0527f68f3a3862afbbfd1b708f1d307d now.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/86762/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    **[Test build #86756 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86756/testReport)** for PR 20404 at commit [`dd1c991`](https://github.com/apache/spark/commit/dd1c9912101389983011b47292da08eff0f43cdc).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/325/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    It’s your call - there are some risks but seems like there are pre existing code that has the same problem.
    
    



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    **[Test build #86759 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86759/testReport)** for PR 20404 at commit [`ec94c05`](https://github.com/apache/spark/commit/ec94c054a8e0abf2d13434556d0434b4690590d3).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Hi all, can you please review again, thanks!


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkS...

Posted by jerryshao <gi...@git.apache.org>.
Github user jerryshao commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20404#discussion_r164350171
  
    --- Diff: python/pyspark/sql/session.py ---
    @@ -760,6 +764,7 @@ def stop(self):
             """Stop the underlying :class:`SparkContext`.
             """
             self._sc.stop()
    +        self._jvm.SparkSession.clearDefaultSession()
    --- End diff --
    
    IIUC, since we already stop the jvm `SparkContext` with above line, it is not necessary to keep it.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkS...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20404#discussion_r164269656
  
    --- Diff: python/pyspark/sql/session.py ---
    @@ -225,6 +225,7 @@ def __init__(self, sparkContext, jsparkSession=None):
             if SparkSession._instantiatedSession is None \
                     or SparkSession._instantiatedSession._sc._jsc is None:
                 SparkSession._instantiatedSession = self
    +            self._jvm.org.apache.spark.sql.SparkSession.setDefaultSession(self._jsparkSession)
    --- End diff --
    
    I think it would overwrite. Can we try something like this?
    
    ```diff
    diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py
    index 6c84023c43f..0bdfc88153f 100644
    --- a/python/pyspark/sql/session.py
    +++ b/python/pyspark/sql/session.py
    @@ -213,7 +213,10 @@ class SparkSession(object):
             self._jsc = self._sc._jsc
             self._jvm = self._sc._jvm
             if jsparkSession is None:
    -            jsparkSession = self._jvm.SparkSession(self._jsc.sc())
    +            if self._jvm.SparkSession.getDefaultSession().isEmpty():
    +                jsparkSession = self._jvm.SparkSession.getDefaultSession().get()
    +            else:
    +                jsparkSession = self._jvm.SparkSession(self._jsc.sc())
             self._jsparkSession = jsparkSession
             self._jwrapped = self._jsparkSession.sqlContext()
             self._wrapped = SQLContext(self._sc, self, self._jwrapped)
    @@ -225,6 +228,8 @@ class SparkSession(object):
             if SparkSession._instantiatedSession is None \
                     or SparkSession._instantiatedSession._sc._jsc is None:
                 SparkSession._instantiatedSession = self
    +            if self._jvm.SparkSession.getDefaultSession().isEmpty():
    +               self._jvm.SparkSession.setDefaultSession(self._jsparkSession)
    
         def _repr_html_(self):
             return """
    @@ -759,6 +764,7 @@ class SparkSession(object):
             """Stop the underlying :class:`SparkContext`.
             """
             self._sc.stop()
    +        self._jvm.org.apache.spark.sql.SparkSession.clearDefaultSession()
             SparkSession._instantiatedSession = None
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/86759/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkS...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20404#discussion_r164088909
  
    --- Diff: python/pyspark/sql/session.py ---
    @@ -225,6 +225,7 @@ def __init__(self, sparkContext, jsparkSession=None):
             if SparkSession._instantiatedSession is None \
                     or SparkSession._instantiatedSession._sc._jsc is None:
                 SparkSession._instantiatedSession = self
    +            self._jvm.org.apache.spark.sql.SparkSession.setDefaultSession(self._jsparkSession)
    --- End diff --
    
    I know it looks a bit odd test but should we better have a test like
    
    ```
    spark._jvm.org.apache.spark.sql.SparkSession.getDefaultSession().nonEmpty()
    ```
    
    ?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    For perfectness, I think we should hold a lock with JVM instance but I wonder if it's easily possible.  I roughly knew this but I think underestimated this because I believe that will quite unlikely happens. I think reverting https://github.com/apache/spark/commit/cc4b8510c1445fb742c0d750958d352adfa84902 doesn't fully resolve the issue because I think the same thing can also happen between `if` and the next line.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    LGTM


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Merged to master.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/86765/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    **[Test build #86760 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86760/testReport)** for PR 20404 at commit [`d9f77ea`](https://github.com/apache/spark/commit/d9f77eaadecedc115dd2dc64843761a3bb08d46e).
     * This patch **fails Python style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    **[Test build #86762 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/86762/testReport)** for PR 20404 at commit [`1ed62ef`](https://github.com/apache/spark/commit/1ed62ef1dbc2c46f2d21eb6147418a86bd59c55c).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by ueshin <gi...@git.apache.org>.
Github user ueshin commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    LGTM.
    I'd like to leave this to @felixcheung to confirm setting the default session is okay or not (https://github.com/apache/spark/pull/20404#discussion_r164362178).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #20404: [SPARK-23228][PYSPARK] Add Python Created jsparkSession ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/20404
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org