[ https://issues.apache.org/jira/browse/SPARK-24523?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16623065#comment-16623065 ]
Imran Rashid commented on SPARK-24523: -------------------------------------- Looks to me like {{SQLAppStatusListener}} is still busy, but the other listeners are fine. In particular, "spark-listener-group-eventLog" seems to have made it through all of the events. That ".inprogress" file is the event log written by the spark application, for the history server to read. I'm not so sure about those lingering threads related to it -- [~ste...@apache.org] maybe you know? I have a feeling their harmless. So really we need to figure out why the SQLAppStatusListener is so far behind for your application. In the first two stack traces, its processing taskEnd events, and the last one is from an ExecutionEnd event. Any chance you can share the event logs? Then we could just try replaying those logs and profiling the SQLAppStatusListener. FYI I probably won't be able to dig into this more in the near term. [~ankur.gupta] maybe interesting for you to look at? > 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: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org