liupengcheng created SPARK-26074:
------------------------------------

             Summary: AsyncEventQueue.stop hangs when eventQueue is full
                 Key: SPARK-26074
                 URL: https://issues.apache.org/jira/browse/SPARK-26074
             Project: Spark
          Issue Type: Bug
          Components: Spark Core
    Affects Versions: 2.4.0, 2.3.0, 2.1.0
            Reporter: liupengcheng


In our production environment, we found a case that the Driver hangs when the 
app finished and about to exit.

Detail information:

 The spark-listener-group-shared Thread may exited due to some plugin 
TaskFailedListener(here in our case is XGBoost),thus the eventQueue might 
easily enter Full state and stay unchanged. After that, if app finished and 
SparkContext.stop is called, the

Driver will hangs at the following stack:
{code:java}
"Driver" #36 prio=5 os_prio=0 tid=0x00007fb08a948800 nid=0x6edc waiting on 
condition [0x00007faff5017000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x00000006c718d7d8> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
at java.util.concurrent.LinkedBlockingQueue.put(LinkedBlockingQueue.java:350)
at org.apache.spark.scheduler.AsyncEventQueue.stop(AsyncEventQueue.scala:117)
at 
org.apache.spark.scheduler.LiveListenerBus$$anonfun$stop$1.apply(LiveListenerBus.scala:202)
at 
org.apache.spark.scheduler.LiveListenerBus$$anonfun$stop$1.apply(LiveListenerBus.scala:202)
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:202)
- locked <0x00000006c052f0a8> (a org.apache.spark.scheduler.LiveListenerBus)
at 
org.apache.spark.SparkContext$$anonfun$stop$6.apply$mcV$sp(SparkContext.scala:1842)
at org.apache.spark.util.Utils$.tryLogNonFatalError(Utils.scala:1294)
at org.apache.spark.SparkContext.stop(SparkContext.scala:1841){code}
Find out it's because spark put a POSION_PILL when AsyncEventQueue.stop, 
however, the queue is in full state, so the put action will be blocked forever.

spark-listener-group-shared exit message:
{code:java}
2018-11-15,14:44:04,782 INFO org.apache.spark.scheduler.AsyncEventQueue: 
Stopping listener queue shared.
java.lang.InterruptedException: ExecutorLost during XGBoost Training: 
TaskKilled (killed intentionally)
at 
org.apache.spark.TaskFailedListener.onTaskEnd(SparkParallelismTracker.scala:116)
at 
org.apache.spark.scheduler.SparkListenerBus$class.doPostEvent(SparkListenerBus.scala:45)
at 
org.apache.spark.scheduler.AsyncEventQueue.doPostEvent(AsyncEventQueue.scala:35)
at 
org.apache.spark.scheduler.AsyncEventQueue.doPostEvent(AsyncEventQueue.scala:35)
at org.apache.spark.util.ListenerBus$class.postToAll(ListenerBus.scala:63)
at 
org.apache.spark.scheduler.AsyncEventQueue.org$apache$spark$scheduler$AsyncEventQueue$$super$postToAll(AsyncEventQueue.scala:83)
at 
org.apache.spark.scheduler.AsyncEventQueue$$anonfun$org$apache$spark$scheduler$AsyncEventQueue$$dispatch$1.apply(AsyncEventQueue.scala:83)
at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
at 
org.apache.spark.scheduler.AsyncEventQueue.org$apache$spark$scheduler$AsyncEventQueue$$dispatch(AsyncEventQueue.scala:79)
at 
org.apache.spark.scheduler.AsyncEventQueue$$anon$1$$anonfun$run$1.apply$mcV$sp(AsyncEventQueue.scala:75)
at org.apache.spark.util.Utils$.tryOrStopSparkContext(Utils.scala:1256)
at 
org.apache.spark.scheduler.AsyncEventQueue$$anon$1.run(AsyncEventQueue.scala:74)
{code}



--
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

Reply via email to