You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "David Lewis (JIRA)" <ji...@apache.org> on 2017/08/04 00:13:02 UTC

[jira] [Commented] (SPARK-19870) Repeatable deadlock on BlockInfoManager and TorrentBroadcast

    [ https://issues.apache.org/jira/browse/SPARK-19870?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16113727#comment-16113727 ] 

David Lewis commented on SPARK-19870:
-------------------------------------

I think I'm hitting a similar bug, here are two stack traces in the block manager, one waiting for read and one waiting for write:
{code}java.lang.Object.wait(Native Method)
java.lang.Object.wait(Object.java:502)
org.apache.spark.storage.BlockInfoManager.lockForWriting(BlockInfoManager.scala:236)
org.apache.spark.storage.BlockManager.removeBlock(BlockManager.scala:1323)
org.apache.spark.storage.BlockManager$$anonfun$removeBroadcast$2.apply(BlockManager.scala:1314)
org.apache.spark.storage.BlockManager$$anonfun$removeBroadcast$2.apply(BlockManager.scala:1314)
scala.collection.Iterator$class.foreach(Iterator.scala:893)
scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
org.apache.spark.storage.BlockManager.removeBroadcast(BlockManager.scala:1314)
org.apache.spark.storage.BlockManagerSlaveEndpoint$$anonfun$receiveAndReply$1$$anonfun$applyOrElse$4.apply$mcI$sp(BlockManagerSlaveEndpoint.scala:66)
org.apache.spark.storage.BlockManagerSlaveEndpoint$$anonfun$receiveAndReply$1$$anonfun$applyOrElse$4.apply(BlockManagerSlaveEndpoint.scala:66)
org.apache.spark.storage.BlockManagerSlaveEndpoint$$anonfun$receiveAndReply$1$$anonfun$applyOrElse$4.apply(BlockManagerSlaveEndpoint.scala:66)
org.apache.spark.storage.BlockManagerSlaveEndpoint$$anonfun$1.apply(BlockManagerSlaveEndpoint.scala:82)
scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24)
scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24)
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617){code}

and 

{code}java.lang.Object.wait(Native Method)
java.lang.Object.wait(Object.java:502)
org.apache.spark.storage.BlockInfoManager.lockForWriting(BlockInfoManager.scala:236)
org.apache.spark.storage.BlockManager.removeBlock(BlockManager.scala:1323)
org.apache.spark.storage.BlockManager$$anonfun$removeBroadcast$2.apply(BlockManager.scala:1314)
org.apache.spark.storage.BlockManager$$anonfun$removeBroadcast$2.apply(BlockManager.scala:1314)
scala.collection.Iterator$class.foreach(Iterator.scala:893)
scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
org.apache.spark.storage.BlockManager.removeBroadcast(BlockManager.scala:1314)
org.apache.spark.storage.BlockManagerSlaveEndpoint$$anonfun$receiveAndReply$1$$anonfun$applyOrElse$4.apply$mcI$sp(BlockManagerSlaveEndpoint.scala:66)
org.apache.spark.storage.BlockManagerSlaveEndpoint$$anonfun$receiveAndReply$1$$anonfun$applyOrElse$4.apply(BlockManagerSlaveEndpoint.scala:66)
org.apache.spark.storage.BlockManagerSlaveEndpoint$$anonfun$receiveAndReply$1$$anonfun$applyOrElse$4.apply(BlockManagerSlaveEndpoint.scala:66)
org.apache.spark.storage.BlockManagerSlaveEndpoint$$anonfun$1.apply(BlockManagerSlaveEndpoint.scala:82)
scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24)
scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24)
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
java.lang.Thread.run(Thread.java:748){code}

> Repeatable deadlock on BlockInfoManager and TorrentBroadcast
> ------------------------------------------------------------
>
>                 Key: SPARK-19870
>                 URL: https://issues.apache.org/jira/browse/SPARK-19870
>             Project: Spark
>          Issue Type: Bug
>          Components: Block Manager, Shuffle
>    Affects Versions: 2.0.2, 2.1.0
>         Environment: ubuntu linux 14.04 x86_64 on ec2, hadoop cdh 5.10.0, yarn coarse-grained.
>            Reporter: Steven Ruppert
>         Attachments: stack.txt
>
>
> Running what I believe to be a fairly vanilla spark job, using the RDD api, with several shuffles, a cached RDD, and finally a conversion to DataFrame to save to parquet. I get a repeatable deadlock at the very last reducers of one of the stages.
> Roughly:
> {noformat}
> "Executor task launch worker-6" #56 daemon prio=5 os_prio=0 tid=0x00007fffd88d3000 nid=0x1022b9 waiting for monitor entry [0x00007fffb95f3000]
>    java.lang.Thread.State: BLOCKED (on object monitor)
>         at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1.apply(TorrentBroadcast.scala:207)
>         - waiting to lock <0x00000005445cfc00> (a org.apache.spark.broadcast.TorrentBroadcast$)
>         at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1269)
>         at org.apache.spark.broadcast.TorrentBroadcast.readBroadcastBlock(TorrentBroadcast.scala:206)
>         at org.apache.spark.broadcast.TorrentBroadcast._value$lzycompute(TorrentBroadcast.scala:66)
>         - locked <0x00000005b12f2290> (a org.apache.spark.broadcast.TorrentBroadcast)
>         at org.apache.spark.broadcast.TorrentBroadcast._value(TorrentBroadcast.scala:66)
>         at org.apache.spark.broadcast.TorrentBroadcast.getValue(TorrentBroadcast.scala:96)
>         at org.apache.spark.broadcast.Broadcast.value(Broadcast.scala:70)
>         at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:86)
>         at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)
>         at org.apache.spark.scheduler.Task.run(Task.scala:99)
>         at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>         at java.lang.Thread.run(Thread.java:745)
> {noformat}
> and 
> {noformat}
> "Executor task launch worker-5" #55 daemon prio=5 os_prio=0 tid=0x00007fffd88d0000 nid=0x1022b8 in Object.wait() [0x00007fffb96f4000]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.Object.wait(Object.java:502)
>         at org.apache.spark.storage.BlockInfoManager.lockForReading(BlockInfoManager.scala:202)
>         - locked <0x0000000545736b58> (a org.apache.spark.storage.BlockInfoManager)
>         at org.apache.spark.storage.BlockManager.getLocalValues(BlockManager.scala:444)
>         at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1.apply(TorrentBroadcast.scala:210)
>         - locked <0x00000005445cfc00> (a org.apache.spark.broadcast.TorrentBroadcast$)
>         at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1269)
>         at org.apache.spark.broadcast.TorrentBroadcast.readBroadcastBlock(TorrentBroadcast.scala:206)
>         at org.apache.spark.broadcast.TorrentBroadcast._value$lzycompute(TorrentBroadcast.scala:66)
>         - locked <0x000000059711eb10> (a org.apache.spark.broadcast.TorrentBroadcast)
>         at org.apache.spark.broadcast.TorrentBroadcast._value(TorrentBroadcast.scala:66)
>         at org.apache.spark.broadcast.TorrentBroadcast.getValue(TorrentBroadcast.scala:96)
>         at org.apache.spark.broadcast.Broadcast.value(Broadcast.scala:70)
>         at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:86)
>         at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)
>         at org.apache.spark.scheduler.Task.run(Task.scala:99)
>         at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>         at java.lang.Thread.run(Thread.java:745)
> {noformat}
> A full stack trace is attached, but those seem to be the offending threads.
> This happens across several different executors, and has persisted through several runs of the same job across spark 2.1.0 and 2.0.2. I also tried killing individual executors to "unstick" the job, to no avail.
> I haven't yet narrowed down the job itself to something publicly repeatable, but hopefully the stacktraces are enough to start debugging.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org