You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by vanzin <gi...@git.apache.org> on 2018/06/25 22:06:10 UTC

[GitHub] spark pull request #21639: [SPARK-24631][tests] Avoid cross-job pollution in...

GitHub user vanzin opened a pull request:

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

    [SPARK-24631][tests] Avoid cross-job pollution in TestUtils / SpillListener.

    There is a narrow race in this code that is caused when the code being
    run in assertSpilled / assertNotSpilled runs more than a single job.
    
    SpillListener assumed that only a single job was run, and so would only
    block waiting for that single job to finish when `numSpilledStages` was
    called. But some tests (like SQL tests that call `checkAnswer`) run more
    than one job, and so that wait was basically a no-op.
    
    This could cause the next test to install a listener to receive events
    from the previous job. Which could cause test failures in certain cases.
    
    The change fixes that race, and also uninstalls listeners after the
    test runs, so they don't accumulate when the SparkContext is shared
    among multiple tests.

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

    $ git pull https://github.com/vanzin/spark SPARK-24653

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

    https://github.com/apache/spark/pull/21639.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 #21639
    
----
commit 9f0a9c4a37f55d0e9591bb59957afcdf745a678d
Author: Marcelo Vanzin <va...@...>
Date:   2018-06-25T21:46:12Z

    [SPARK-24631][tests] Avoid cross-job pollution in TestUtils / SpillListener.
    
    There is a narrow race in this code that is caused when the code being
    run in assertSpilled / assertNotSpilled runs more than a single job.
    
    SpillListener assumed that only a single job was run, and so would only
    block waiting for that single job to finish when `numSpilledStages` was
    called. But some tests (like SQL tests that call `checkAnswer`) run more
    than one job, and so that wait was basically a no-op.
    
    This could cause the next test to install a listener to receive events
    from the previous job. Which could cause test failures in certain cases.
    
    The change fixes that race, and also uninstalls listeners after the
    test runs, so they don't accumulate when the SparkContext is shared
    among multiple tests.

----


---

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


[GitHub] spark issue #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    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 #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    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 #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    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-unified/1544/
    Test PASSed.


---

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


[GitHub] spark issue #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    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 #21639: [SPARK-24631][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    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 #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    **[Test build #93091 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93091/testReport)** for PR 21639 at commit [`18d5ebf`](https://github.com/apache/spark/commit/18d5ebfd201deaebf774835ec5eb08d2b6d08454).


---

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


[GitHub] spark issue #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    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 #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    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 #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    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 #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    **[Test build #93861 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93861/testReport)** for PR 21639 at commit [`18d5ebf`](https://github.com/apache/spark/commit/18d5ebfd201deaebf774835ec5eb08d2b6d08454).


---

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


[GitHub] spark pull request #21639: [SPARK-24631][tests] Avoid cross-job pollution in...

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

    https://github.com/apache/spark/pull/21639#discussion_r198188319
  
    --- Diff: core/src/main/scala/org/apache/spark/TestUtils.scala ---
    @@ -173,21 +173,23 @@ private[spark] object TestUtils {
        * Run some code involving jobs submitted to the given context and assert that the jobs spilled.
        */
       def assertSpilled[T](sc: SparkContext, identifier: String)(body: => T): Unit = {
    -    val spillListener = new SpillListener
    -    sc.addSparkListener(spillListener)
    -    body
    -    assert(spillListener.numSpilledStages > 0, s"expected $identifier to spill, but did not")
    +    withListener(sc, new SpillListener) { listener =>
    +      val ret = body
    --- End diff --
    
    Maybe I'm missing something obvious, but why shall we need the return value here?


---

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


[GitHub] spark issue #21639: [SPARK-24631][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    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 #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    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-unified/981/
    Test PASSed.


---

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


[GitHub] spark issue #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

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


---

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


[GitHub] spark issue #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    **[Test build #92345 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92345/testReport)** for PR 21639 at commit [`18d5ebf`](https://github.com/apache/spark/commit/18d5ebfd201deaebf774835ec5eb08d2b6d08454).
     * 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 #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    **[Test build #4226 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/4226/testReport)** for PR 21639 at commit [`18d5ebf`](https://github.com/apache/spark/commit/18d5ebfd201deaebf774835ec5eb08d2b6d08454).


---

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


[GitHub] spark issue #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    retest this please


---

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


[GitHub] spark pull request #21639: [SPARK-24631][tests] Avoid cross-job pollution in...

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

    https://github.com/apache/spark/pull/21639#discussion_r198186779
  
    --- Diff: core/src/main/scala/org/apache/spark/TestUtils.scala ---
    @@ -173,21 +173,23 @@ private[spark] object TestUtils {
        * Run some code involving jobs submitted to the given context and assert that the jobs spilled.
        */
       def assertSpilled[T](sc: SparkContext, identifier: String)(body: => T): Unit = {
    -    val spillListener = new SpillListener
    -    sc.addSparkListener(spillListener)
    -    body
    -    assert(spillListener.numSpilledStages > 0, s"expected $identifier to spill, but did not")
    +    withListener(sc, new SpillListener) { listener =>
    +      val ret = body
    +      assert(listener.numSpilledStages > 0, s"expected $identifier to spill, but did not")
    --- End diff --
    
    Maybe I'm missing something obvious, but shouldn't we run the `body` before assert?


---

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


[GitHub] spark pull request #21639: [SPARK-24653][tests] Avoid cross-job pollution in...

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

    https://github.com/apache/spark/pull/21639#discussion_r206697727
  
    --- Diff: core/src/main/scala/org/apache/spark/TestUtils.scala ---
    @@ -233,6 +233,21 @@ private[spark] object TestUtils {
         }
       }
     
    +  /**
    +   * Runs some code with the given listener installed in the SparkContext. After the code runs,
    +   * this method will wait until all events posted to the listener bus are processed, and then
    +   * remove the listener from the bus.
    +   */
    +  def withListener[L <: SparkListener](sc: SparkContext, listener: L) (body: L => Unit): Unit = {
    --- End diff --
    
    private? hardly matters.


---

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


[GitHub] spark issue #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

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


---

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


[GitHub] spark issue #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    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 #21639: [SSPARK-24653][tests] Avoid cross-job pollution in TestU...

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

    https://github.com/apache/spark/pull/21639
  
    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 #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    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 #21639: [SSPARK-24653][tests] Avoid cross-job pollution in TestU...

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

    https://github.com/apache/spark/pull/21639
  
    Oops, no idea how I got the wrong bug.


---

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


[GitHub] spark issue #21639: [SPARK-24631][tests] Avoid cross-job pollution in TestUt...

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

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


---

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


[GitHub] spark issue #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    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 #21639: [SPARK-24631][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    Seems the JIRA number is not related?


---

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


[GitHub] spark issue #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    **[Test build #93185 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93185/testReport)** for PR 21639 at commit [`18d5ebf`](https://github.com/apache/spark/commit/18d5ebfd201deaebf774835ec5eb08d2b6d08454).


---

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


[GitHub] spark issue #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    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-unified/1060/
    Test PASSed.


---

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


[GitHub] spark issue #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

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


---

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


[GitHub] spark issue #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    **[Test build #4226 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/4226/testReport)** for PR 21639 at commit [`18d5ebf`](https://github.com/apache/spark/commit/18d5ebfd201deaebf774835ec5eb08d2b6d08454).
     * This patch **fails Spark unit tests**.
     * This patch **does not merge 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 #21639: [SPARK-24631][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    **[Test build #92312 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92312/testReport)** for PR 21639 at commit [`9f0a9c4`](https://github.com/apache/spark/commit/9f0a9c4a37f55d0e9591bb59957afcdf745a678d).
     * 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 #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    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-unified/1001/
    Test PASSed.


---

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


[GitHub] spark issue #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    lgtm


---

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


[GitHub] spark issue #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    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 #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    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 #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    **[Test build #93091 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93091/testReport)** for PR 21639 at commit [`18d5ebf`](https://github.com/apache/spark/commit/18d5ebfd201deaebf774835ec5eb08d2b6d08454).
     * This patch **fails Spark 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 pull request #21639: [SPARK-24653][tests] Avoid cross-job pollution in...

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

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


---

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


[GitHub] spark issue #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    **[Test build #93861 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93861/testReport)** for PR 21639 at commit [`18d5ebf`](https://github.com/apache/spark/commit/18d5ebfd201deaebf774835ec5eb08d2b6d08454).
     * 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 #21639: [SPARK-24631][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    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-unified/465/
    Test PASSed.


---

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


[GitHub] spark issue #21639: [SSPARK-24653][tests] Avoid cross-job pollution in TestU...

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

    https://github.com/apache/spark/pull/21639
  
    **[Test build #92345 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92345/testReport)** for PR 21639 at commit [`18d5ebf`](https://github.com/apache/spark/commit/18d5ebfd201deaebf774835ec5eb08d2b6d08454).


---

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


[GitHub] spark issue #21639: [SPARK-24631][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    **[Test build #92312 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92312/testReport)** for PR 21639 at commit [`9f0a9c4`](https://github.com/apache/spark/commit/9f0a9c4a37f55d0e9591bb59957afcdf745a678d).


---

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


[GitHub] spark pull request #21639: [SPARK-24653][tests] Avoid cross-job pollution in...

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

    https://github.com/apache/spark/pull/21639#discussion_r198210412
  
    --- Diff: core/src/main/scala/org/apache/spark/TestUtils.scala ---
    @@ -173,21 +173,23 @@ private[spark] object TestUtils {
        * Run some code involving jobs submitted to the given context and assert that the jobs spilled.
        */
       def assertSpilled[T](sc: SparkContext, identifier: String)(body: => T): Unit = {
    -    val spillListener = new SpillListener
    -    sc.addSparkListener(spillListener)
    -    body
    -    assert(spillListener.numSpilledStages > 0, s"expected $identifier to spill, but did not")
    +    withListener(sc, new SpillListener) { listener =>
    +      val ret = body
    --- End diff --
    
    I saw the return type in the closure, but the method itself returns `Unit`, so all that can be cleaned up.


---

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


[GitHub] spark issue #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    **[Test build #93063 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93063/testReport)** for PR 21639 at commit [`18d5ebf`](https://github.com/apache/spark/commit/18d5ebfd201deaebf774835ec5eb08d2b6d08454).


---

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


[GitHub] spark issue #21639: [SSPARK-24653][tests] Avoid cross-job pollution in TestU...

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

    https://github.com/apache/spark/pull/21639
  
    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-unified/486/
    Test PASSed.


---

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


[GitHub] spark issue #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

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


---

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


[GitHub] spark issue #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

    https://github.com/apache/spark/pull/21639
  
    **[Test build #93063 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93063/testReport)** for PR 21639 at commit [`18d5ebf`](https://github.com/apache/spark/commit/18d5ebfd201deaebf774835ec5eb08d2b6d08454).
     * 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 issue #21639: [SPARK-24653][tests] Avoid cross-job pollution in TestUt...

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

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


---

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