Skip to content

Conversation

@tdas
Copy link
Contributor

@tdas tdas commented May 21, 2015

Shutdown hook to stop SparkContext was added recently. This results in ugly errors when a streaming application is terminated by ctrl-C.

Exception in thread "Thread-27" org.apache.spark.SparkException: Job cancelled because SparkContext was shut down
    at org.apache.spark.scheduler.DAGScheduler$$anonfun$cleanUpAfterSchedulerStop$1.apply(DAGScheduler.scala:736)
    at org.apache.spark.scheduler.DAGScheduler$$anonfun$cleanUpAfterSchedulerStop$1.apply(DAGScheduler.scala:735)
    at scala.collection.mutable.HashSet.foreach(HashSet.scala:79)
    at org.apache.spark.scheduler.DAGScheduler.cleanUpAfterSchedulerStop(DAGScheduler.scala:735)
    at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onStop(DAGScheduler.scala:1468)
    at org.apache.spark.util.EventLoop.stop(EventLoop.scala:84)
    at org.apache.spark.scheduler.DAGScheduler.stop(DAGScheduler.scala:1403)
    at org.apache.spark.SparkContext.stop(SparkContext.scala:1642)
    at org.apache.spark.SparkContext$$anonfun$3.apply$mcV$sp(SparkContext.scala:559)
    at org.apache.spark.util.SparkShutdownHook.run(Utils.scala:2266)
    at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(Utils.scala:2236)
    at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(Utils.scala:2236)
    at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(Utils.scala:2236)
    at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1764)
    at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply$mcV$sp(Utils.scala:2236)
    at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply(Utils.scala:2236)
    at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply(Utils.scala:2236)
    at scala.util.Try$.apply(Try.scala:161)
    at org.apache.spark.util.SparkShutdownHookManager.runAll(Utils.scala:2236)
    at org.apache.spark.util.SparkShutdownHookManager$$anon$6.run(Utils.scala:2218)
    at org.apache.hadoop.util.ShutdownHookManager$1.run(ShutdownHookManager.java:54)

This is because the Spark's shutdown hook stops the context, and the streaming jobs fail in the middle. The correct solution is to stop the streaming context before the spark context. This PR adds the shutdown hook to do so with a priority higher than the SparkContext's shutdown hooks priority.

@tdas
Copy link
Contributor Author

tdas commented May 21, 2015

@zsxwing Another PR for you to review.

@zsxwing
Copy link
Member

zsxwing commented May 21, 2015

LGTM

@SparkQA
Copy link

SparkQA commented May 21, 2015

Test build #33195 has finished for PR 6307 at commit 4c18652.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@tdas
Copy link
Contributor Author

tdas commented May 21, 2015

@zsxwing Can you take a look again. Made a small change.

@harishreedharan
Copy link
Contributor

+1.

@zsxwing
Copy link
Member

zsxwing commented May 21, 2015

@tdas as per javadoc: https://docs.oracle.com/javase/7/docs/api/java/lang/Runtime.html#addShutdownHook(java.lang.Thread)

Shutdown hooks should also finish their work quickly. When a program invokes exit the expectation is that the virtual machine will promptly shut down and exit. When the virtual machine is terminated due to user logoff or system shutdown the underlying operating system may only allow a fixed amount of time in which to shut down and exit. It is therefore inadvisable to attempt any user interaction or to perform a long-running computation in a shutdown hook.

I think maybe we should not support stopping gracefully in the shutdown hook.

@SparkQA
Copy link

SparkQA commented May 21, 2015

Test build #33208 has finished for PR 6307 at commit e3d5475.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@tdas
Copy link
Contributor Author

tdas commented May 21, 2015

@zsxwing I completely agree. That's why I am not sure about adding this and wanted to get more feedback. The reason for attempting this was this thread - https://www.mail-archive.com/user@spark.apache.org/msg28788.html

Your thoughts?

@tdas
Copy link
Contributor Author

tdas commented May 22, 2015

@zsxwing I chatted with @pwendell about this and he thinks its okay to have that. By default it will exit fast as graceful stop is disabled. If someone wants to enable it, it will be at their own risk.

@zsxwing
Copy link
Member

zsxwing commented May 22, 2015

I see. LGTM

@asfgit asfgit closed this in d68ea24 May 22, 2015
asfgit pushed a commit that referenced this pull request May 22, 2015
Shutdown hook to stop SparkContext was added recently. This results in ugly errors when a streaming application is terminated by ctrl-C.

```
Exception in thread "Thread-27" org.apache.spark.SparkException: Job cancelled because SparkContext was shut down
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$cleanUpAfterSchedulerStop$1.apply(DAGScheduler.scala:736)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$cleanUpAfterSchedulerStop$1.apply(DAGScheduler.scala:735)
	at scala.collection.mutable.HashSet.foreach(HashSet.scala:79)
	at org.apache.spark.scheduler.DAGScheduler.cleanUpAfterSchedulerStop(DAGScheduler.scala:735)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onStop(DAGScheduler.scala:1468)
	at org.apache.spark.util.EventLoop.stop(EventLoop.scala:84)
	at org.apache.spark.scheduler.DAGScheduler.stop(DAGScheduler.scala:1403)
	at org.apache.spark.SparkContext.stop(SparkContext.scala:1642)
	at org.apache.spark.SparkContext$$anonfun$3.apply$mcV$sp(SparkContext.scala:559)
	at org.apache.spark.util.SparkShutdownHook.run(Utils.scala:2266)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(Utils.scala:2236)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(Utils.scala:2236)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(Utils.scala:2236)
	at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1764)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply$mcV$sp(Utils.scala:2236)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply(Utils.scala:2236)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply(Utils.scala:2236)
	at scala.util.Try$.apply(Try.scala:161)
	at org.apache.spark.util.SparkShutdownHookManager.runAll(Utils.scala:2236)
	at org.apache.spark.util.SparkShutdownHookManager$$anon$6.run(Utils.scala:2218)
	at org.apache.hadoop.util.ShutdownHookManager$1.run(ShutdownHookManager.java:54)
```

This is because the Spark's shutdown hook stops the context, and the streaming jobs fail in the middle. The correct solution is to stop the streaming context before the spark context. This PR adds the shutdown hook to do so with a priority higher than the SparkContext's shutdown hooks priority.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #6307 from tdas/SPARK-7776 and squashes the following commits:

e3d5475 [Tathagata Das] Added conf to specify graceful shutdown
4c18652 [Tathagata Das] Added shutdown hook to StreamingContxt.

(cherry picked from commit d68ea24)
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
jeanlyn pushed a commit to jeanlyn/spark that referenced this pull request May 28, 2015
Shutdown hook to stop SparkContext was added recently. This results in ugly errors when a streaming application is terminated by ctrl-C.

```
Exception in thread "Thread-27" org.apache.spark.SparkException: Job cancelled because SparkContext was shut down
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$cleanUpAfterSchedulerStop$1.apply(DAGScheduler.scala:736)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$cleanUpAfterSchedulerStop$1.apply(DAGScheduler.scala:735)
	at scala.collection.mutable.HashSet.foreach(HashSet.scala:79)
	at org.apache.spark.scheduler.DAGScheduler.cleanUpAfterSchedulerStop(DAGScheduler.scala:735)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onStop(DAGScheduler.scala:1468)
	at org.apache.spark.util.EventLoop.stop(EventLoop.scala:84)
	at org.apache.spark.scheduler.DAGScheduler.stop(DAGScheduler.scala:1403)
	at org.apache.spark.SparkContext.stop(SparkContext.scala:1642)
	at org.apache.spark.SparkContext$$anonfun$3.apply$mcV$sp(SparkContext.scala:559)
	at org.apache.spark.util.SparkShutdownHook.run(Utils.scala:2266)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(Utils.scala:2236)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(Utils.scala:2236)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(Utils.scala:2236)
	at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1764)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply$mcV$sp(Utils.scala:2236)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply(Utils.scala:2236)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply(Utils.scala:2236)
	at scala.util.Try$.apply(Try.scala:161)
	at org.apache.spark.util.SparkShutdownHookManager.runAll(Utils.scala:2236)
	at org.apache.spark.util.SparkShutdownHookManager$$anon$6.run(Utils.scala:2218)
	at org.apache.hadoop.util.ShutdownHookManager$1.run(ShutdownHookManager.java:54)
```

This is because the Spark's shutdown hook stops the context, and the streaming jobs fail in the middle. The correct solution is to stop the streaming context before the spark context. This PR adds the shutdown hook to do so with a priority higher than the SparkContext's shutdown hooks priority.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes apache#6307 from tdas/SPARK-7776 and squashes the following commits:

e3d5475 [Tathagata Das] Added conf to specify graceful shutdown
4c18652 [Tathagata Das] Added shutdown hook to StreamingContxt.
jeanlyn pushed a commit to jeanlyn/spark that referenced this pull request Jun 12, 2015
Shutdown hook to stop SparkContext was added recently. This results in ugly errors when a streaming application is terminated by ctrl-C.

```
Exception in thread "Thread-27" org.apache.spark.SparkException: Job cancelled because SparkContext was shut down
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$cleanUpAfterSchedulerStop$1.apply(DAGScheduler.scala:736)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$cleanUpAfterSchedulerStop$1.apply(DAGScheduler.scala:735)
	at scala.collection.mutable.HashSet.foreach(HashSet.scala:79)
	at org.apache.spark.scheduler.DAGScheduler.cleanUpAfterSchedulerStop(DAGScheduler.scala:735)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onStop(DAGScheduler.scala:1468)
	at org.apache.spark.util.EventLoop.stop(EventLoop.scala:84)
	at org.apache.spark.scheduler.DAGScheduler.stop(DAGScheduler.scala:1403)
	at org.apache.spark.SparkContext.stop(SparkContext.scala:1642)
	at org.apache.spark.SparkContext$$anonfun$3.apply$mcV$sp(SparkContext.scala:559)
	at org.apache.spark.util.SparkShutdownHook.run(Utils.scala:2266)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(Utils.scala:2236)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(Utils.scala:2236)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(Utils.scala:2236)
	at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1764)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply$mcV$sp(Utils.scala:2236)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply(Utils.scala:2236)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply(Utils.scala:2236)
	at scala.util.Try$.apply(Try.scala:161)
	at org.apache.spark.util.SparkShutdownHookManager.runAll(Utils.scala:2236)
	at org.apache.spark.util.SparkShutdownHookManager$$anon$6.run(Utils.scala:2218)
	at org.apache.hadoop.util.ShutdownHookManager$1.run(ShutdownHookManager.java:54)
```

This is because the Spark's shutdown hook stops the context, and the streaming jobs fail in the middle. The correct solution is to stop the streaming context before the spark context. This PR adds the shutdown hook to do so with a priority higher than the SparkContext's shutdown hooks priority.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes apache#6307 from tdas/SPARK-7776 and squashes the following commits:

e3d5475 [Tathagata Das] Added conf to specify graceful shutdown
4c18652 [Tathagata Das] Added shutdown hook to StreamingContxt.
nemccarthy pushed a commit to nemccarthy/spark that referenced this pull request Jun 19, 2015
Shutdown hook to stop SparkContext was added recently. This results in ugly errors when a streaming application is terminated by ctrl-C.

```
Exception in thread "Thread-27" org.apache.spark.SparkException: Job cancelled because SparkContext was shut down
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$cleanUpAfterSchedulerStop$1.apply(DAGScheduler.scala:736)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$cleanUpAfterSchedulerStop$1.apply(DAGScheduler.scala:735)
	at scala.collection.mutable.HashSet.foreach(HashSet.scala:79)
	at org.apache.spark.scheduler.DAGScheduler.cleanUpAfterSchedulerStop(DAGScheduler.scala:735)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onStop(DAGScheduler.scala:1468)
	at org.apache.spark.util.EventLoop.stop(EventLoop.scala:84)
	at org.apache.spark.scheduler.DAGScheduler.stop(DAGScheduler.scala:1403)
	at org.apache.spark.SparkContext.stop(SparkContext.scala:1642)
	at org.apache.spark.SparkContext$$anonfun$3.apply$mcV$sp(SparkContext.scala:559)
	at org.apache.spark.util.SparkShutdownHook.run(Utils.scala:2266)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(Utils.scala:2236)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(Utils.scala:2236)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(Utils.scala:2236)
	at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1764)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply$mcV$sp(Utils.scala:2236)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply(Utils.scala:2236)
	at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply(Utils.scala:2236)
	at scala.util.Try$.apply(Try.scala:161)
	at org.apache.spark.util.SparkShutdownHookManager.runAll(Utils.scala:2236)
	at org.apache.spark.util.SparkShutdownHookManager$$anon$6.run(Utils.scala:2218)
	at org.apache.hadoop.util.ShutdownHookManager$1.run(ShutdownHookManager.java:54)
```

This is because the Spark's shutdown hook stops the context, and the streaming jobs fail in the middle. The correct solution is to stop the streaming context before the spark context. This PR adds the shutdown hook to do so with a priority higher than the SparkContext's shutdown hooks priority.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes apache#6307 from tdas/SPARK-7776 and squashes the following commits:

e3d5475 [Tathagata Das] Added conf to specify graceful shutdown
4c18652 [Tathagata Das] Added shutdown hook to StreamingContxt.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants