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 2015/04/01 19:02:32 UTC

[GitHub] spark pull request: [SPARK-6650] [core] Stop ExecutorAllocationMan...

GitHub user vanzin opened a pull request:

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

    [SPARK-6650] [core] Stop ExecutorAllocationManager when context stops.

    This fixes the thread leak. I also changed the unit test to keep track
    of allocated contexts and make sure they're closed after tests are
    run; this is needed since some tests use this pattern:
    
        val sc = createContext()
        doSomethingThatMayThrow()
        sc.stop()

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

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

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

    https://github.com/apache/spark/pull/5311.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 #5311
    
----
commit 9886f697bc3ab6fe83aa2aa4ba8a3e92ae0ec051
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2015-04-01T16:49:47Z

    [SPARK-6650] [core] Stop ExecutorAllocationManager when context stops.
    
    This fixes the thread leak. I also changed the unit test to keep track
    of allocated contexts and making sure they're closed after tests are
    run; this is needed since some tests use this pattern:
    
        val sc = createContext()
        doSomethingThatMayThrow()
        sc.stop()

----


---
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-6650] [core] Stop ExecutorAllocationMan...

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

    https://github.com/apache/spark/pull/5311#issuecomment-88590553
  
    Jenkins retest this please.


---
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-6650] [core] Stop ExecutorAllocationMan...

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

    https://github.com/apache/spark/pull/5311#issuecomment-88558037
  
      [Test build #29548 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/29548/consoleFull) for   PR 5311 at commit [`9886f69`](https://github.com/apache/spark/commit/9886f697bc3ab6fe83aa2aa4ba8a3e92ae0ec051).


---
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-6650] [core] Stop ExecutorAllocationMan...

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

    https://github.com/apache/spark/pull/5311#issuecomment-88604118
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/29551/
    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-6650] [core] Stop ExecutorAllocationMan...

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

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


---
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-6650] [core] Stop ExecutorAllocationMan...

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

    https://github.com/apache/spark/pull/5311#issuecomment-88587640
  
      [Test build #29548 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/29548/consoleFull) for   PR 5311 at commit [`9886f69`](https://github.com/apache/spark/commit/9886f697bc3ab6fe83aa2aa4ba8a3e92ae0ec051).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.
     * This patch does not change any dependencies.


---
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-6650] [core] Stop ExecutorAllocationMan...

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

    https://github.com/apache/spark/pull/5311#issuecomment-89135509
  
    LGTM2, thanks for fixing this merging into master and 1.3.


---
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-6650] [core] Stop ExecutorAllocationMan...

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

    https://github.com/apache/spark/pull/5311#issuecomment-88604111
  
      [Test build #29551 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/29551/consoleFull) for   PR 5311 at commit [`cc5a744`](https://github.com/apache/spark/commit/cc5a7441a8726af9f8a3ac8a04883172a107c109).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.
     * This patch does not change any dependencies.


---
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-6650] [core] Stop ExecutorAllocationMan...

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

    https://github.com/apache/spark/pull/5311#discussion_r27611599
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -17,10 +17,12 @@
     
     package org.apache.spark
     
    +import java.util.concurrent.{TimeUnit, Executors}
    --- End diff --
    
    nit: alphabetize


---
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-6650] [core] Stop ExecutorAllocationMan...

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

    https://github.com/apache/spark/pull/5311#issuecomment-88592320
  
      [Test build #29554 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/29554/consoleFull) for   PR 5311 at commit [`cc5a744`](https://github.com/apache/spark/commit/cc5a7441a8726af9f8a3ac8a04883172a107c109).


---
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-6650] [core] Stop ExecutorAllocationMan...

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

    https://github.com/apache/spark/pull/5311#discussion_r27590605
  
    --- Diff: core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala ---
    @@ -28,10 +28,20 @@ import org.apache.spark.util.ManualClock
     /**
      * Test add and remove behavior of ExecutorAllocationManager.
      */
    -class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
    +class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext with BeforeAndAfter {
       import ExecutorAllocationManager._
       import ExecutorAllocationManagerSuite._
     
    +  private val contexts = new mutable.ListBuffer[SparkContext]()
    --- End diff --
    
    Yes. See ` test("verify min/max executors")`.



---
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-6650] [core] Stop ExecutorAllocationMan...

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

    https://github.com/apache/spark/pull/5311#issuecomment-88631842
  
    One last nit.  Otherwise LGTM.


---
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-6650] [core] Stop ExecutorAllocationMan...

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

    https://github.com/apache/spark/pull/5311#issuecomment-88587669
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/29548/
    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-6650] [core] Stop ExecutorAllocationMan...

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

    https://github.com/apache/spark/pull/5311#issuecomment-88618922
  
    I'm not sure why the test failed, but on inspection it LGTM.


---
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-6650] [core] Stop ExecutorAllocationMan...

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

    https://github.com/apache/spark/pull/5311#issuecomment-88571273
  
      [Test build #29551 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/29551/consoleFull) for   PR 5311 at commit [`cc5a744`](https://github.com/apache/spark/commit/cc5a7441a8726af9f8a3ac8a04883172a107c109).


---
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-6650] [core] Stop ExecutorAllocationMan...

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

    https://github.com/apache/spark/pull/5311#issuecomment-88635291
  
      [Test build #29564 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/29564/consoleFull) for   PR 5311 at commit [`652c73b`](https://github.com/apache/spark/commit/652c73b7d35ab80f84d11879c7d5f1d6c613d8cd).


---
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-6650] [core] Stop ExecutorAllocationMan...

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

    https://github.com/apache/spark/pull/5311#issuecomment-88611637
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/29554/
    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-6650] [core] Stop ExecutorAllocationMan...

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

    https://github.com/apache/spark/pull/5311#discussion_r27590421
  
    --- Diff: core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala ---
    @@ -28,10 +28,20 @@ import org.apache.spark.util.ManualClock
     /**
      * Test add and remove behavior of ExecutorAllocationManager.
      */
    -class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
    +class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext with BeforeAndAfter {
       import ExecutorAllocationManager._
       import ExecutorAllocationManagerSuite._
     
    +  private val contexts = new mutable.ListBuffer[SparkContext]()
    --- End diff --
    
    Will this ever have more than a single element?


---
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-6650] [core] Stop ExecutorAllocationMan...

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

    https://github.com/apache/spark/pull/5311#issuecomment-88611617
  
      [Test build #29554 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/29554/consoleFull) for   PR 5311 at commit [`cc5a744`](https://github.com/apache/spark/commit/cc5a7441a8726af9f8a3ac8a04883172a107c109).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.
     * This patch does not change any dependencies.


---
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-6650] [core] Stop ExecutorAllocationMan...

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

    https://github.com/apache/spark/pull/5311#discussion_r27591507
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -173,32 +179,24 @@ private[spark] class ExecutorAllocationManager(
       }
     
       /**
    -   * Register for scheduler callbacks to decide when to add and remove executors.
    +   * Register for scheduler callbacks to decide when to add and remove executors, and start
    +   * the scheduling task.
        */
       def start(): Unit = {
         listenerBus.addListener(listener)
    -    startPolling()
    +
    +    val scheduleTask = new Runnable() {
    +      override def run(): Unit = Utils.logUncaughtExceptions(schedule())
    +    }
    +    executor.scheduleAtFixedRate(scheduleTask, 0, intervalMillis, TimeUnit.MILLISECONDS)
       }
     
       /**
    -   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * Stop the allocation manager.
        */
    -  private def startPolling(): Unit = {
    -    val t = new Thread {
    -      override def run(): Unit = {
    -        while (true) {
    -          try {
    -            schedule()
    -          } catch {
    -            case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    -          }
    -          Thread.sleep(intervalMillis)
    -        }
    -      }
    -    }
    -    t.setName("spark-dynamic-executor-allocation")
    -    t.setDaemon(true)
    -    t.start()
    +  def stop(): Unit = {
    +    executor.shutdown()
    +    executor.awaitTermination(10, TimeUnit.SECONDS)
    --- End diff --
    
    Thinking aloud here:
    We stop the ExecutorAllocationManager after stopping the DAGScheduler, which means, in yarn-client mode, we'll have already torn down the YARN application.  If `schedule` is called after that, we could be trying to make an RPC to the AM, which is no longer there.  So it seems like waiting the full 10 seconds for that RPC to time out could be a common case.


---
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-6650] [core] Stop ExecutorAllocationMan...

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

    https://github.com/apache/spark/pull/5311#issuecomment-88655484
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/29564/
    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-6650] [core] Stop ExecutorAllocationMan...

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

    https://github.com/apache/spark/pull/5311#discussion_r27591780
  
    --- Diff: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala ---
    @@ -173,32 +179,24 @@ private[spark] class ExecutorAllocationManager(
       }
     
       /**
    -   * Register for scheduler callbacks to decide when to add and remove executors.
    +   * Register for scheduler callbacks to decide when to add and remove executors, and start
    +   * the scheduling task.
        */
       def start(): Unit = {
         listenerBus.addListener(listener)
    -    startPolling()
    +
    +    val scheduleTask = new Runnable() {
    +      override def run(): Unit = Utils.logUncaughtExceptions(schedule())
    +    }
    +    executor.scheduleAtFixedRate(scheduleTask, 0, intervalMillis, TimeUnit.MILLISECONDS)
       }
     
       /**
    -   * Start the main polling thread that keeps track of when to add and remove executors.
    +   * Stop the allocation manager.
        */
    -  private def startPolling(): Unit = {
    -    val t = new Thread {
    -      override def run(): Unit = {
    -        while (true) {
    -          try {
    -            schedule()
    -          } catch {
    -            case e: Exception => logError("Exception in dynamic executor allocation thread!", e)
    -          }
    -          Thread.sleep(intervalMillis)
    -        }
    -      }
    -    }
    -    t.setName("spark-dynamic-executor-allocation")
    -    t.setDaemon(true)
    -    t.start()
    +  def stop(): Unit = {
    +    executor.shutdown()
    +    executor.awaitTermination(10, TimeUnit.SECONDS)
    --- End diff --
    
    I tried to follow the order of initialization during shutdown (the alloc manager is started after the event logger so should be stopped right before it - yeah, I know my code doesn't do exactly that).
    
    But we could move the stop before the scheduler's stop.


---
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-6650] [core] Stop ExecutorAllocationMan...

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

    https://github.com/apache/spark/pull/5311#issuecomment-88655474
  
      [Test build #29564 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/29564/consoleFull) for   PR 5311 at commit [`652c73b`](https://github.com/apache/spark/commit/652c73b7d35ab80f84d11879c7d5f1d6c613d8cd).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.
     * This patch does not change any dependencies.


---
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