You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by ted-yu <gi...@git.apache.org> on 2015/11/08 18:11:42 UTC

[GitHub] spark pull request: Exit AsynchronousListenerBus thread when stop(...

GitHub user ted-yu opened a pull request:

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

    Exit AsynchronousListenerBus thread when stop() is called

    As vonnagy reported in the following thread:
    http://search-hadoop.com/m/q3RTtk982kvIow22
    
    Attempts to join the thread in AsynchronousListenerBus resulted in lock up because AsynchronousListenerBus thread was still getting messages `SparkListenerExecutorMetricsUpdate` from the DAGScheduler

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

    $ git pull https://github.com/ted-yu/spark master

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

    https://github.com/apache/spark/pull/9546.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 #9546
    
----
commit c60b8607201ac7094f39499439fe5022d0e26de3
Author: tedyu <yu...@gmail.com>
Date:   2015-11-08T17:08:54Z

    Exit AsynchronousListenerBus thread when stop() is called

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

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

    https://github.com/apache/spark/pull/9546#issuecomment-155255884
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: Exit AsynchronousListenerBus thread when stop(...

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

    https://github.com/apache/spark/pull/9546#issuecomment-154841878
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

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

    https://github.com/apache/spark/pull/9546#issuecomment-155257790
  
    **[Test build #45464 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45464/consoleFull)** for PR 9546 at commit [`9acce70`](https://github.com/apache/spark/commit/9acce70366eef89175a7789780c54af1d79b43ab).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

Posted by vonnagy <gi...@git.apache.org>.
Github user vonnagy commented on the pull request:

    https://github.com/apache/spark/pull/9546#issuecomment-155166356
  
    I was unable to duplicate the issue I had with the `listenerThread.join` so it seems that your change solved the issue. 
    
    If I encounter the issue again then I can create a PR to adjust the `listenerThread.join()` accordingly.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

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

    https://github.com/apache/spark/pull/9546#discussion_r44353658
  
    --- Diff: core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala ---
    @@ -66,6 +66,7 @@ private[spark] abstract class AsynchronousListenerBus[L <: AnyRef, E](name: Stri
             self.synchronized {
               processingEvent = true
             }
    +        if (stopped.get()) return
    --- End diff --
    
    instead of doing this here, can you simplify the try block as follows?
    ```
    try {
      if (stopped.get()) { return }
      val event = eventQueue.poll()
      assert(event != null, "event queue was empty but the listener bus was not stopped")
      postToAll(event)
    } finally {
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

Posted by ted-yu <gi...@git.apache.org>.
Github user ted-yu commented on the pull request:

    https://github.com/apache/spark/pull/9546#issuecomment-156866484
  
    Planning to send out a PR to fix the regression by keeping count of queued events first time seeing the stop flag.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

Posted by ted-yu <gi...@git.apache.org>.
Github user ted-yu commented on the pull request:

    https://github.com/apache/spark/pull/9546#issuecomment-155276724
  
    I see several errors in the following form (https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45460/consoleFull):
    ```
    [error] SERVER ERROR: Service Temporarily Unavailable url=http://repository.mapr.com/maven/org/apache/hadoop/hadoop-yarn-server/2.2.0/hadoop-yarn-server-2.2.0.jar
    ```
    Not related to the PR.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

Posted by ted-yu <gi...@git.apache.org>.
Github user ted-yu commented on the pull request:

    https://github.com/apache/spark/pull/9546#issuecomment-156867526
  
    Cloning git repo was extremely slow.
    Here is proposed fix:
    ```
    diff --git a/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala b/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala
    index b3b54af..cc58bc5 100644
    --- a/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala
    +++ b/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala
    @@ -56,19 +56,24 @@ private[spark] abstract class AsynchronousListenerBus[L <: AnyRef, E](name: Stri
    
       // A counter that represents the number of events produced and consumed in the queue
       private val eventLock = new Semaphore(0)
    +  // limit on the number of events to process before exiting. -1 means no limit
    +  private val eventLimit = -1
    
       private val listenerThread = new Thread(name) {
         setDaemon(true)
         override def run(): Unit = Utils.tryOrStopSparkContext(sparkContext) {
    -      while (true) {
    +      while (eventLimit != 0) {
             eventLock.acquire()
             self.synchronized {
               processingEvent = true
             }
             try {
               if (stopped.get()) {
    -            // Get out of the while loop and shutdown the daemon thread
    -            return
    +            eventLimit = eventQueue.size
    +            if (eventLimit == 0) {
    +              // Get out of the while loop and shutdown the daemon thread
    +              return
    +            }
               }
               val event = eventQueue.poll
               assert(event != null, "event queue was empty but the listener bus was not stopped")
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

Posted by vonnagy <gi...@git.apache.org>.
Github user vonnagy commented on the pull request:

    https://github.com/apache/spark/pull/9546#issuecomment-154882409
  
    @ted-yu I have found that the call to `listener.join` (line 168) sometimes still locks up. I have changed the code slightly to use something like `listener.join(20)` (line 168) so that if the thread does not exit it will give it a quick break before trying again.
    
    Would you mind adding something like that to your PR?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/9546#issuecomment-156846877
  
    I think that this has caused the  "org.apache.spark.scheduler.EventLoggingListenerSuite.End-to-end event logging" test to become flaky in Jenkins.
    
    I believe that this patch may have changed the behavior of the listener bus during shutdown. According to the `stop()` method's Scaladoc:
    
    ```
      /**
       * Stop the listener bus. It will wait until the queued events have been processed, but drop the
       * new events after stopping.
       */
    ```
    
    It looks like this patch just changes things so that we halt immediately once the `stopped` flag has been set rather than waiting for the queue to drain.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

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

    https://github.com/apache/spark/pull/9546#issuecomment-155275929
  
    **[Test build #45460 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45460/consoleFull)** for PR 9546 at commit [`9a6f9ff`](https://github.com/apache/spark/commit/9a6f9ffa7fc1c39cd337cfa1eda1bf65eca85879).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

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

    https://github.com/apache/spark/pull/9546#issuecomment-155280584
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

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

    https://github.com/apache/spark/pull/9546#issuecomment-155275985
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

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

    https://github.com/apache/spark/pull/9546#issuecomment-155257702
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

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

    https://github.com/apache/spark/pull/9546#issuecomment-154858927
  
    **[Test build #45302 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45302/consoleFull)** for PR 9546 at commit [`c60b860`](https://github.com/apache/spark/commit/c60b8607201ac7094f39499439fe5022d0e26de3).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

Posted by ted-yu <gi...@git.apache.org>.
Github user ted-yu commented on the pull request:

    https://github.com/apache/spark/pull/9546#issuecomment-156850070
  
    I checked 
    https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-Maven-with-YARN/HADOOP_PROFILE=hadoop-2.4,label=spark-test/4122/consoleFull
    back till:
    https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-Maven-with-YARN/HADOOP_PROFILE=hadoop-2.4,label=spark-test/4118/consoleFull
    
    https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-Maven-with-YARN/HADOOP_PROFILE=hadoop-2.3,label=spark-test/4122/consoleFull
    back until:
    https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-Maven-with-YARN/HADOOP_PROFILE=hadoop-2.3,label=spark-test/4119/consoleFull
    
    EventLoggingListenerSuite passed in every build above



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

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

    https://github.com/apache/spark/pull/9546#discussion_r44361026
  
    --- Diff: core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala ---
    @@ -66,16 +66,13 @@ private[spark] abstract class AsynchronousListenerBus[L <: AnyRef, E](name: Stri
             self.synchronized {
               processingEvent = true
             }
    +        if (stopped.get()) {
    +          // Get out of the while loop and shutdown the daemon thread
    +          return
    +        }
    --- End diff --
    
    can you move this into the `try`? If `stopped.get` throws an exception we still want to set `processingEvent` to false


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

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

    https://github.com/apache/spark/pull/9546#issuecomment-155257683
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

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

    https://github.com/apache/spark/pull/9546#issuecomment-155255863
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

Posted by ted-yu <gi...@git.apache.org>.
Github user ted-yu commented on the pull request:

    https://github.com/apache/spark/pull/9546#issuecomment-155533447
  
    @andrewor14 
    See if all comments have been addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

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

    https://github.com/apache/spark/pull/9546#issuecomment-154858958
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

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

    https://github.com/apache/spark/pull/9546#issuecomment-155256025
  
    **[Test build #45460 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45460/consoleFull)** for PR 9546 at commit [`9a6f9ff`](https://github.com/apache/spark/commit/9a6f9ffa7fc1c39cd337cfa1eda1bf65eca85879).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

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

    https://github.com/apache/spark/pull/9546#issuecomment-155280457
  
    **[Test build #45464 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45464/consoleFull)** for PR 9546 at commit [`9acce70`](https://github.com/apache/spark/commit/9acce70366eef89175a7789780c54af1d79b43ab).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: Exit AsynchronousListenerBus thread when stop(...

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

    https://github.com/apache/spark/pull/9546#issuecomment-154842290
  
    **[Test build #45302 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45302/consoleFull)** for PR 9546 at commit [`c60b860`](https://github.com/apache/spark/commit/c60b8607201ac7094f39499439fe5022d0e26de3).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: Exit AsynchronousListenerBus thread when stop(...

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

    https://github.com/apache/spark/pull/9546#issuecomment-154841886
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11572] Exit AsynchronousListenerBus thr...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/9546#issuecomment-156851581
  
    Look at the Master SBT build; there's definitely a regression: https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-SBT/4014/AMPLAB_JENKINS_BUILD_PROFILE=hadoop2.3,label=spark-test/testReport/junit/org.apache.spark.scheduler/EventLoggingListenerSuite/End_to_end_event_logging/history/
    
    If you keep clicking on the "Older" link to page back through the test history, you'll find that this first started in https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-SBT/AMPLAB_JENKINS_BUILD_PROFILE=hadoop2.3,label=spark-test/3982/testReport/, whose changeset includes this patch: https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-SBT/AMPLAB_JENKINS_BUILD_PROFILE=hadoop2.3,label=spark-test/3982/changes


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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