[
https://issues.apache.org/jira/browse/SPARK-24523?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16623357#comment-16623357
]
Steve Loughran commented on SPARK-24523:
----------------------------------------
This sounds like a near replica of (HADOOP-15679, SPARK-25183)
# event logs are renamed in shutdown
# for a real (tm) filesystem, that's an O(1) operation
# for object stores it can be one of: O(1): gcs, O(1)+sometimes O(data), azure,
and O(data) always: S3.
The bigger the event log, the longer the time to rename.
This causes the timeout code of HADOOP-12950 to conclude after 10s that the
shutdown hook has hung, and forcibly interrupt it.
Fixes:
you'll need to talk to the AWS team here. Ideally S3 would add an O(1) rename
operation and we would all be so very happy, but more realistically, if they
are using a fork of hadoop 2.8 underneath, they'll need to backport
HADOOP-15679 into it.
Which, looking at, I have yet to merge with branch-2 for various reasons. I'll
wrap that up into the ASF 2.8 branch, where maybe they'll pick it up themselves.
bq. Don't get why there're in-progress blocks when the application was done
writing all Parquet files more than an hour ago.
rename() is the enemy. More specifically, the way object store connectors mimic
rename, and applications use it because in a real FS it and rm are the two
atomic operations you get to play with. For object stores, [PUT is the new
rename()|https://www.slideshare.net/steve_l/put-is-the-new-rename-san-jose-summit-edition]
> InterruptedException when closing SparkContext
> ----------------------------------------------
>
> Key: SPARK-24523
> URL: https://issues.apache.org/jira/browse/SPARK-24523
> Project: Spark
> Issue Type: Bug
> Components: Scheduler
> Affects Versions: 2.3.0, 2.3.1
> Environment: EMR 5.14.0, S3/HDFS inputs and outputs; EMR 5.17
>
>
>
> Reporter: Umayr Hassan
> Priority: Major
> Attachments: spark-stop-jstack.log.1, spark-stop-jstack.log.2,
> spark-stop-jstack.log.3
>
>
> I'm running a Scala application in EMR with the following properties:
> {{--master yarn --deploy-mode cluster --driver-memory 13g --executor-memory
> 30g --executor-cores 5 --conf spark.default.parallelism=400 --conf
> spark.dynamicAllocation.enabled=true --conf
> spark.dynamicAllocation.maxExecutors=20 --conf
> spark.eventLog.dir=hdfs:///var/log/spark/apps --conf
> spark.eventLog.enabled=true --conf
> spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version=2 --conf
> spark.scheduler.listenerbus.eventqueue.capacity=20000 --conf
> spark.shuffle.service.enabled=true --conf spark.sql.shuffle.partitions=400
> --conf spark.yarn.maxAppAttempts=1}}
> The application runs fine till SparkContext is (automatically) closed, at
> which point the SparkContext object throws.
> {{18/06/10 10:44:43 ERROR Utils: Uncaught exception in thread pool-4-thread-1
> java.lang.InterruptedException at java.lang.Object.wait(Native Method) at
> java.lang.Thread.join(Thread.java:1252) at
> java.lang.Thread.join(Thread.java:1326) at
> org.apache.spark.scheduler.AsyncEventQueue.stop(AsyncEventQueue.scala:133) at
> org.apache.spark.scheduler.LiveListenerBus$$anonfun$stop$1.apply(LiveListenerBus.scala:219)
> at
> org.apache.spark.scheduler.LiveListenerBus$$anonfun$stop$1.apply(LiveListenerBus.scala:219)
> at scala.collection.Iterator$class.foreach(Iterator.scala:893) at
> scala.collection.AbstractIterator.foreach(Iterator.scala:1336) at
> scala.collection.IterableLike$class.foreach(IterableLike.scala:72) at
> scala.collection.AbstractIterable.foreach(Iterable.scala:54) at
> org.apache.spark.scheduler.LiveListenerBus.stop(LiveListenerBus.scala:219) at
> org.apache.spark.SparkContext$$anonfun$stop$6.apply$mcV$sp(SparkContext.scala:1915)
> at org.apache.spark.util.Utils$.tryLogNonFatalError(Utils.scala:1357) at
> org.apache.spark.SparkContext.stop(SparkContext.scala:1914) at
> org.apache.spark.SparkContext$$anonfun$2.apply$mcV$sp(SparkContext.scala:572)
> 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:1988) 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)}}
>
> I've not seen this behavior in Spark 2.0.2 and Spark 2.2.0 (for the same
> application), so I'm not sure which change is causing Spark 2.3 to throw. Any
> ideas?
> best,
> Umayr
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]