Re: Spark Streaming graceful shutdown in Spark 1.4
My 2 cents: 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. The shutdown hook should not do any long-running work and may exit before stop returns. It means we cannot implement the stopGracefully = true semantics correctly, which the user will expect stops gracefully by waiting for the processing of all received data to be completed. So I agree that we can add `ssc.stop` as a the shutdown hook. But stopGracefully should be false. Best Regards, Shixiong Zhu 2015-05-20 21:59 GMT-07:00 Dibyendu Bhattacharya dibyendu.bhattach...@gmail.com: Thanks Tathagata for making this change.. Dibyendu On Thu, May 21, 2015 at 8:24 AM, Tathagata Das t...@databricks.com wrote: If you are talking about handling driver crash failures, then all bets are off anyways! Adding a shutdown hook in the hope of handling driver process failure, handles only a some cases (Ctrl-C), but does not handle cases like SIGKILL (does not run JVM shutdown hooks) or driver machine crash. So its not a good idea to rely on that. Nonetheless I have opened a PR to handle the shutdown of the StreamigntContext in the same way as SparkContext. https://github.com/apache/spark/pull/6307 On Tue, May 19, 2015 at 12:51 AM, Dibyendu Bhattacharya dibyendu.bhattach...@gmail.com wrote: Thenka Sean . you are right. If driver program is running then I can handle shutdown in main exit path . But if Driver machine is crashed (if you just stop the application, for example killing the driver process ), then Shutdownhook is the only option isn't it ? What I try to say is , just doing ssc.stop in sys.ShutdownHookThread or Runtime.getRuntime().addShutdownHook ( in java) wont work anymore. I need to use the Utils.addShutdownHook with a priority .. So just checking if Spark Streaming can make graceful shutdown as default shutdown mechanism. Dibyendu On Tue, May 19, 2015 at 1:03 PM, Sean Owen so...@cloudera.com wrote: I don't think you should rely on a shutdown hook. Ideally you try to stop it in the main exit path of your program, even in case of an exception. On Tue, May 19, 2015 at 7:59 AM, Dibyendu Bhattacharya dibyendu.bhattach...@gmail.com wrote: You mean to say within Runtime.getRuntime().addShutdownHook I call ssc.stop(stopSparkContext = true, stopGracefully = true) ? This won't work anymore in 1.4. The SparkContext got stopped before Receiver processed all received blocks and I see below exception in logs. But if I add the Utils.addShutdownHook with the priority as I mentioned , then only graceful shutdown works . In that case shutdown-hook run in priority order.
Re: Spark Streaming graceful shutdown in Spark 1.4
If you are talking about handling driver crash failures, then all bets are off anyways! Adding a shutdown hook in the hope of handling driver process failure, handles only a some cases (Ctrl-C), but does not handle cases like SIGKILL (does not run JVM shutdown hooks) or driver machine crash. So its not a good idea to rely on that. Nonetheless I have opened a PR to handle the shutdown of the StreamigntContext in the same way as SparkContext. https://github.com/apache/spark/pull/6307 On Tue, May 19, 2015 at 12:51 AM, Dibyendu Bhattacharya dibyendu.bhattach...@gmail.com wrote: Thenka Sean . you are right. If driver program is running then I can handle shutdown in main exit path . But if Driver machine is crashed (if you just stop the application, for example killing the driver process ), then Shutdownhook is the only option isn't it ? What I try to say is , just doing ssc.stop in sys.ShutdownHookThread or Runtime.getRuntime().addShutdownHook ( in java) wont work anymore. I need to use the Utils.addShutdownHook with a priority .. So just checking if Spark Streaming can make graceful shutdown as default shutdown mechanism. Dibyendu On Tue, May 19, 2015 at 1:03 PM, Sean Owen so...@cloudera.com wrote: I don't think you should rely on a shutdown hook. Ideally you try to stop it in the main exit path of your program, even in case of an exception. On Tue, May 19, 2015 at 7:59 AM, Dibyendu Bhattacharya dibyendu.bhattach...@gmail.com wrote: You mean to say within Runtime.getRuntime().addShutdownHook I call ssc.stop(stopSparkContext = true, stopGracefully = true) ? This won't work anymore in 1.4. The SparkContext got stopped before Receiver processed all received blocks and I see below exception in logs. But if I add the Utils.addShutdownHook with the priority as I mentioned , then only graceful shutdown works . In that case shutdown-hook run in priority order.
Re: Spark Streaming graceful shutdown in Spark 1.4
Thanks Tathagata for making this change.. Dibyendu On Thu, May 21, 2015 at 8:24 AM, Tathagata Das t...@databricks.com wrote: If you are talking about handling driver crash failures, then all bets are off anyways! Adding a shutdown hook in the hope of handling driver process failure, handles only a some cases (Ctrl-C), but does not handle cases like SIGKILL (does not run JVM shutdown hooks) or driver machine crash. So its not a good idea to rely on that. Nonetheless I have opened a PR to handle the shutdown of the StreamigntContext in the same way as SparkContext. https://github.com/apache/spark/pull/6307 On Tue, May 19, 2015 at 12:51 AM, Dibyendu Bhattacharya dibyendu.bhattach...@gmail.com wrote: Thenka Sean . you are right. If driver program is running then I can handle shutdown in main exit path . But if Driver machine is crashed (if you just stop the application, for example killing the driver process ), then Shutdownhook is the only option isn't it ? What I try to say is , just doing ssc.stop in sys.ShutdownHookThread or Runtime.getRuntime().addShutdownHook ( in java) wont work anymore. I need to use the Utils.addShutdownHook with a priority .. So just checking if Spark Streaming can make graceful shutdown as default shutdown mechanism. Dibyendu On Tue, May 19, 2015 at 1:03 PM, Sean Owen so...@cloudera.com wrote: I don't think you should rely on a shutdown hook. Ideally you try to stop it in the main exit path of your program, even in case of an exception. On Tue, May 19, 2015 at 7:59 AM, Dibyendu Bhattacharya dibyendu.bhattach...@gmail.com wrote: You mean to say within Runtime.getRuntime().addShutdownHook I call ssc.stop(stopSparkContext = true, stopGracefully = true) ? This won't work anymore in 1.4. The SparkContext got stopped before Receiver processed all received blocks and I see below exception in logs. But if I add the Utils.addShutdownHook with the priority as I mentioned , then only graceful shutdown works . In that case shutdown-hook run in priority order.
Re: Spark Streaming graceful shutdown in Spark 1.4
) at scala.Option.foreach(Option.scala:236) at org.apache.spark.streaming.scheduler.ReceivedBlockTracker.writeToLog(ReceivedBlockTracker.scala:218) at org.apache.spark.streaming.scheduler.ReceivedBlockTracker.allocateBlocksToBatch(ReceivedBlockTracker.scala:108) at org.apache.spark.streaming.scheduler.ReceiverTracker.allocateBlocksToBatch(ReceiverTracker.scala:105) at org.apache.spark.streaming.scheduler.JobGenerator$$anonfun$2.apply(JobGenerator.scala:242) at org.apache.spark.streaming.scheduler.JobGenerator$$anonfun$2.apply(JobGenerator.scala:241) at scala.util.Try$.apply(Try.scala:161) at org.apache.spark.streaming.scheduler.JobGenerator.generateJobs(JobGenerator.scala:241) at org.apache.spark.streaming.scheduler.JobGenerator.org $apache$spark$streaming$scheduler$JobGenerator$$processEvent(JobGenerator.scala:177) at org.apache.spark.streaming.scheduler.JobGenerator$$anon$1.onReceive(JobGenerator.scala:83) at org.apache.spark.streaming.scheduler.JobGenerator$$anon$1.onReceive(JobGenerator.scala:82) at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48) Exception in thread main java.lang.IllegalStateException: Shutdown in progress On Tue, May 19, 2015 at 11:58 AM, Tathagata Das t...@databricks.com wrote: If you wanted to stop it gracefully, then why are you not calling ssc.stop(stopGracefully = true, stopSparkContext = true)? Then it doesnt matter whether the shutdown hook was called or not. TD On Mon, May 18, 2015 at 9:43 PM, Dibyendu Bhattacharya dibyendu.bhattach...@gmail.com wrote: Hi, Just figured out that if I want to perform graceful shutdown of Spark Streaming 1.4 ( from master ) , the Runtime.getRuntime().addShutdownHook no longer works . As in Spark 1.4 there is Utils.addShutdownHook defined for Spark Core, that gets anyway called , which leads to graceful shutdown from Spark streaming failed with error like Sparkcontext already closed issue. To solve this , I need to explicitly add Utils.addShutdownHook in my driver with higher priority ( say 150 ) than Spark's shutdown priority of 50 , and there I specified streamingcontext stop method with (false , true) parameter. Just curious to know , if this is how we need to handle shutdown hook going forward ? Can't we make the streaming shutdown default to gracefully shutdown ? Also the Java Api for adding shutdownhook in Utils looks very dirty with methods like this .. Utils.addShutdownHook(150, new Function0BoxedUnit() { @Override public BoxedUnit apply() { return null; } @Override public byte apply$mcB$sp() { return 0; } @Override public char apply$mcC$sp() { return 0; } @Override public double apply$mcD$sp() { return 0; } @Override public float apply$mcF$sp() { return 0; } @Override public int apply$mcI$sp() { // TODO Auto-generated method stub return 0; } @Override public long apply$mcJ$sp() { return 0; } @Override public short apply$mcS$sp() { return 0; } @Override public void apply$mcV$sp() { *jsc.stop(false, true);* } @Override public boolean apply$mcZ$sp() { // TODO Auto-generated method stub return false; } });
Re: Spark Streaming graceful shutdown in Spark 1.4
I don't think you should rely on a shutdown hook. Ideally you try to stop it in the main exit path of your program, even in case of an exception. On Tue, May 19, 2015 at 7:59 AM, Dibyendu Bhattacharya dibyendu.bhattach...@gmail.com wrote: You mean to say within Runtime.getRuntime().addShutdownHook I call ssc.stop(stopSparkContext = true, stopGracefully = true) ? This won't work anymore in 1.4. The SparkContext got stopped before Receiver processed all received blocks and I see below exception in logs. But if I add the Utils.addShutdownHook with the priority as I mentioned , then only graceful shutdown works . In that case shutdown-hook run in priority order. - To unsubscribe, e-mail: user-unsubscr...@spark.apache.org For additional commands, e-mail: user-h...@spark.apache.org
Re: Spark Streaming graceful shutdown in Spark 1.4
Thenka Sean . you are right. If driver program is running then I can handle shutdown in main exit path . But if Driver machine is crashed (if you just stop the application, for example killing the driver process ), then Shutdownhook is the only option isn't it ? What I try to say is , just doing ssc.stop in sys.ShutdownHookThread or Runtime.getRuntime().addShutdownHook ( in java) wont work anymore. I need to use the Utils.addShutdownHook with a priority .. So just checking if Spark Streaming can make graceful shutdown as default shutdown mechanism. Dibyendu On Tue, May 19, 2015 at 1:03 PM, Sean Owen so...@cloudera.com wrote: I don't think you should rely on a shutdown hook. Ideally you try to stop it in the main exit path of your program, even in case of an exception. On Tue, May 19, 2015 at 7:59 AM, Dibyendu Bhattacharya dibyendu.bhattach...@gmail.com wrote: You mean to say within Runtime.getRuntime().addShutdownHook I call ssc.stop(stopSparkContext = true, stopGracefully = true) ? This won't work anymore in 1.4. The SparkContext got stopped before Receiver processed all received blocks and I see below exception in logs. But if I add the Utils.addShutdownHook with the priority as I mentioned , then only graceful shutdown works . In that case shutdown-hook run in priority order.
Re: Spark Streaming graceful shutdown in Spark 1.4
If you wanted to stop it gracefully, then why are you not calling ssc.stop(stopGracefully = true, stopSparkContext = true)? Then it doesnt matter whether the shutdown hook was called or not. TD On Mon, May 18, 2015 at 9:43 PM, Dibyendu Bhattacharya dibyendu.bhattach...@gmail.com wrote: Hi, Just figured out that if I want to perform graceful shutdown of Spark Streaming 1.4 ( from master ) , the Runtime.getRuntime().addShutdownHook no longer works . As in Spark 1.4 there is Utils.addShutdownHook defined for Spark Core, that gets anyway called , which leads to graceful shutdown from Spark streaming failed with error like Sparkcontext already closed issue. To solve this , I need to explicitly add Utils.addShutdownHook in my driver with higher priority ( say 150 ) than Spark's shutdown priority of 50 , and there I specified streamingcontext stop method with (false , true) parameter. Just curious to know , if this is how we need to handle shutdown hook going forward ? Can't we make the streaming shutdown default to gracefully shutdown ? Also the Java Api for adding shutdownhook in Utils looks very dirty with methods like this .. Utils.addShutdownHook(150, new Function0BoxedUnit() { @Override public BoxedUnit apply() { return null; } @Override public byte apply$mcB$sp() { return 0; } @Override public char apply$mcC$sp() { return 0; } @Override public double apply$mcD$sp() { return 0; } @Override public float apply$mcF$sp() { return 0; } @Override public int apply$mcI$sp() { // TODO Auto-generated method stub return 0; } @Override public long apply$mcJ$sp() { return 0; } @Override public short apply$mcS$sp() { return 0; } @Override public void apply$mcV$sp() { *jsc.stop(false, true);* } @Override public boolean apply$mcZ$sp() { // TODO Auto-generated method stub return false; } });
Re: Spark Streaming graceful shutdown in Spark 1.4
(ReceiverTracker.scala:105) at org.apache.spark.streaming.scheduler.JobGenerator$$anonfun$2.apply(JobGenerator.scala:242) at org.apache.spark.streaming.scheduler.JobGenerator$$anonfun$2.apply(JobGenerator.scala:241) at scala.util.Try$.apply(Try.scala:161) at org.apache.spark.streaming.scheduler.JobGenerator.generateJobs(JobGenerator.scala:241) at org.apache.spark.streaming.scheduler.JobGenerator.org $apache$spark$streaming$scheduler$JobGenerator$$processEvent(JobGenerator.scala:177) at org.apache.spark.streaming.scheduler.JobGenerator$$anon$1.onReceive(JobGenerator.scala:83) at org.apache.spark.streaming.scheduler.JobGenerator$$anon$1.onReceive(JobGenerator.scala:82) at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48) Exception in thread main java.lang.IllegalStateException: Shutdown in progress On Tue, May 19, 2015 at 11:58 AM, Tathagata Das t...@databricks.com wrote: If you wanted to stop it gracefully, then why are you not calling ssc.stop(stopGracefully = true, stopSparkContext = true)? Then it doesnt matter whether the shutdown hook was called or not. TD On Mon, May 18, 2015 at 9:43 PM, Dibyendu Bhattacharya dibyendu.bhattach...@gmail.com wrote: Hi, Just figured out that if I want to perform graceful shutdown of Spark Streaming 1.4 ( from master ) , the Runtime.getRuntime().addShutdownHook no longer works . As in Spark 1.4 there is Utils.addShutdownHook defined for Spark Core, that gets anyway called , which leads to graceful shutdown from Spark streaming failed with error like Sparkcontext already closed issue. To solve this , I need to explicitly add Utils.addShutdownHook in my driver with higher priority ( say 150 ) than Spark's shutdown priority of 50 , and there I specified streamingcontext stop method with (false , true) parameter. Just curious to know , if this is how we need to handle shutdown hook going forward ? Can't we make the streaming shutdown default to gracefully shutdown ? Also the Java Api for adding shutdownhook in Utils looks very dirty with methods like this .. Utils.addShutdownHook(150, new Function0BoxedUnit() { @Override public BoxedUnit apply() { return null; } @Override public byte apply$mcB$sp() { return 0; } @Override public char apply$mcC$sp() { return 0; } @Override public double apply$mcD$sp() { return 0; } @Override public float apply$mcF$sp() { return 0; } @Override public int apply$mcI$sp() { // TODO Auto-generated method stub return 0; } @Override public long apply$mcJ$sp() { return 0; } @Override public short apply$mcS$sp() { return 0; } @Override public void apply$mcV$sp() { *jsc.stop(false, true);* } @Override public boolean apply$mcZ$sp() { // TODO Auto-generated method stub return false; } });
Spark Streaming graceful shutdown in Spark 1.4
Hi, Just figured out that if I want to perform graceful shutdown of Spark Streaming 1.4 ( from master ) , the Runtime.getRuntime().addShutdownHook no longer works . As in Spark 1.4 there is Utils.addShutdownHook defined for Spark Core, that gets anyway called , which leads to graceful shutdown from Spark streaming failed with error like Sparkcontext already closed issue. To solve this , I need to explicitly add Utils.addShutdownHook in my driver with higher priority ( say 150 ) than Spark's shutdown priority of 50 , and there I specified streamingcontext stop method with (false , true) parameter. Just curious to know , if this is how we need to handle shutdown hook going forward ? Can't we make the streaming shutdown default to gracefully shutdown ? Also the Java Api for adding shutdownhook in Utils looks very dirty with methods like this .. Utils.addShutdownHook(150, new Function0BoxedUnit() { @Override public BoxedUnit apply() { return null; } @Override public byte apply$mcB$sp() { return 0; } @Override public char apply$mcC$sp() { return 0; } @Override public double apply$mcD$sp() { return 0; } @Override public float apply$mcF$sp() { return 0; } @Override public int apply$mcI$sp() { // TODO Auto-generated method stub return 0; } @Override public long apply$mcJ$sp() { return 0; } @Override public short apply$mcS$sp() { return 0; } @Override public void apply$mcV$sp() { *jsc.stop(false, true);* } @Override public boolean apply$mcZ$sp() { // TODO Auto-generated method stub return false; } });