Details
-
Bug
-
Status: Resolved
-
Major
-
Resolution: Incomplete
-
2.2.0, 2.2.1, 2.2.2, 2.3.0, 2.3.1
-
None
-
Spark Streaming
– Tested on 2.2 & 2.3 (more than likely affects all versions with graceful shutdown)
Hadoop 2.8
Description
Opening this up to give you guys some insight in an issue that will occur when using Spark Streaming with Hadoop 2.8.
Hadoop 2.8 added HADOOP-12950 which adds a upper limit of a 10 second timeout for its shutdown hook. From our tests, if the Spark job takes longer than 10 seconds to shutdown gracefully, the Hadoop shutdown thread seems to trample over the graceful shutdown and throw an exception like
18/08/03 17:21:04 ERROR Utils: Uncaught exception in thread pool-1-thread-1 java.lang.InterruptedException at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(AbstractQueuedSynchronizer.java:1039) at java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1328) at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:277) at org.apache.spark.streaming.scheduler.ReceiverTracker.stop(ReceiverTracker.scala:177) at org.apache.spark.streaming.scheduler.JobScheduler.stop(JobScheduler.scala:114) at org.apache.spark.streaming.StreamingContext$$anonfun$stop$1.apply$mcV$sp(StreamingContext.scala:682) at org.apache.spark.util.Utils$.tryLogNonFatalError(Utils.scala:1317) at org.apache.spark.streaming.StreamingContext.stop(StreamingContext.scala:681) at org.apache.spark.streaming.StreamingContext.org$apache$spark$streaming$StreamingContext$$stopOnShutdown(StreamingContext.scala:715) at org.apache.spark.streaming.StreamingContext$$anonfun$start$1.apply$mcV$sp(StreamingContext.scala:599) at org.apache.spark.util.SparkShutdownHook.run(ShutdownHookManager.scala:216) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(ShutdownHookManager.scala:188) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(ShutdownHookManager.scala:188) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(ShutdownHookManager.scala:188) at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1948) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply$mcV$sp(ShutdownHookManager.scala:188) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply(ShutdownHookManager.scala:188) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply(ShutdownHookManager.scala:188) at scala.util.Try$.apply(Try.scala:192) at org.apache.spark.util.SparkShutdownHookManager.runAll(ShutdownHookManager.scala:188) at org.apache.spark.util.SparkShutdownHookManager$$anon$2.run(ShutdownHookManager.scala:178) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748)
The reason I hit this issue is because we recently upgraded to EMR 5.15, which has both Spark 2.3 & Hadoop 2.8. The following workaround has proven successful to us (in limited testing)
Instead of just running
... ssc.start() ssc.awaitTermination()
We needed to do the following
... ssc.start() sys.ShutdownHookThread { ssc.stop(true, true) } ssc.awaitTermination()
As far as I can tell, there is no way to override the default 10 second timeout in HADOOP-12950, which is why we had to go with the workaround.
Note: I also verified this bug exists even with EMR 5.12.1 which runs Spark 2.2.x & Hadoop 2.8.
Ricky
Epic Games