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

[jira] [Assigned] (SPARK-27338) Deadlock between TaskMemoryManager and UnsafeExternalSorter$SpillableIterator

     [ https://issues.apache.org/jira/browse/SPARK-27338?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Wenchen Fan reassigned SPARK-27338:
-----------------------------------

    Assignee: Venkata krishnan Sowrirajan

> Deadlock between TaskMemoryManager and UnsafeExternalSorter$SpillableIterator
> -----------------------------------------------------------------------------
>
>                 Key: SPARK-27338
>                 URL: https://issues.apache.org/jira/browse/SPARK-27338
>             Project: Spark
>          Issue Type: Bug
>          Components: Spark Core
>    Affects Versions: 2.4.0
>            Reporter: Venkata krishnan Sowrirajan
>            Assignee: Venkata krishnan Sowrirajan
>            Priority: Major
>
> We saw similar deadlock like this https://issues.apache.org/jira/browse/SPARK-26265 happening between TaskMemoryManager and UnsafeExternalSorted$SpillableIterator
> Jstack output:
> jstack information as follow:
> {code:java}
> Found one Java-level deadlock:
> =============================
> "stdout writer for /usr/lib/envs/env-1923-ver-1755-a-4.2.9-py-3.5.3/bin/python":
>   waiting to lock monitor 0x00007fce56409088 (object 0x00000005700a2f98, a org.apache.spark.memory.TaskMemoryManager),
>   which is held by "Executor task launch worker for task 2203"
> "Executor task launch worker for task 2203":
>   waiting to lock monitor 0x00000000007cd878 (object 0x00000005701a0eb0, a org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter$SpillableIterator),
>   which is held by "stdout writer for /usr/lib/envs/env-1923-ver-1755-a-4.2.9-py-3.5.3/bin/python"
> Java stack information for the threads listed above:
> ===================================================
> "stdout writer for /usr/lib/envs/env-1923-ver-1755-a-4.2.9-py-3.5.3/bin/python":
> 	at org.apache.spark.memory.TaskMemoryManager.freePage(TaskMemoryManager.java:334)
> 	- waiting to lock <0x00000005700a2f98> (a org.apache.spark.memory.TaskMemoryManager)
> 	at org.apache.spark.memory.MemoryConsumer.freePage(MemoryConsumer.java:130)
> 	at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.access$1100(UnsafeExternalSorter.java:48)
> 	at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter$SpillableIterator.loadNext(UnsafeExternalSorter.java:583)
> 	- locked <0x00000005701a0eb0> (a org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter$SpillableIterator)
> 	at org.apache.spark.sql.execution.UnsafeExternalRowSorter$1.next(UnsafeExternalRowSorter.java:187)
> 	at org.apache.spark.sql.execution.UnsafeExternalRowSorter$1.next(UnsafeExternalRowSorter.java:174)
> 	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage3.processNext(Unknown Source)
> 	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
> 	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$11$$anon$1.hasNext(WholeStageCodegenExec.scala:619)
> 	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage10.findNextInnerJoinRows$(Unknown Source)
> 	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage10.processNext(Unknown Source)
> 	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
> 	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$13$$anon$2.hasNext(WholeStageCodegenExec.scala:638)
> 	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:409)
> 	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:409)
> 	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:409)
> 	at scala.collection.Iterator$GroupedIterator.takeDestructively(Iterator.scala:1073)
> 	at scala.collection.Iterator$GroupedIterator.go(Iterator.scala:1089)
> 	at scala.collection.Iterator$GroupedIterator.fill(Iterator.scala:1127)
> 	at scala.collection.Iterator$GroupedIterator.hasNext(Iterator.scala:1130)
> 	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:409)
> 	at scala.collection.Iterator$class.foreach(Iterator.scala:891)
> 	at scala.collection.AbstractIterator.foreach(Iterator.scala:1334)
> 	at org.apache.spark.api.python.PythonRDD$.writeIteratorToStream(PythonRDD.scala:224)
> 	at org.apache.spark.sql.execution.python.PythonUDFRunner$$anon$2.writeIteratorToStream(PythonUDFRunner.scala:50)
> 	at org.apache.spark.api.python.BasePythonRunner$WriterThread$$anonfun$run$1.apply(PythonRunner.scala:345)
> 	at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:2067)
> 	at org.apache.spark.api.python.BasePythonRunner$WriterThread.run(PythonRunner.scala:194)
> "Executor task launch worker for task 2203":
> 	at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter$SpillableIterator.spill(UnsafeExternalSorter.java:525)
> 	- waiting to lock <0x00000005701a0eb0> (a org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter$SpillableIterator)
> 	at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.spill(UnsafeExternalSorter.java:200)
> 	at org.apache.spark.memory.TaskMemoryManager.acquireExecutionMemory(TaskMemoryManager.java:177)
> 	- locked <0x00000005700a2f98> (a org.apache.spark.memory.TaskMemoryManager)
> 	at org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:285)
> 	at org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:117)
> 	at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPageIfNecessary(UnsafeExternalSorter.java:383)
> 	at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertRecord(UnsafeExternalSorter.java:407)
> 	at org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:135)
> 	at org.apache.spark.sql.execution.UnsafeExternalRowSorter.sort(UnsafeExternalRowSorter.java:217)
> 	at org.apache.spark.sql.execution.SortExec$$anonfun$1.apply(SortExec.scala:108)
> 	at org.apache.spark.sql.execution.SortExec$$anonfun$1.apply(SortExec.scala:101)
> 	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:836)
> 	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:836)
> 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
> 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
> 	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
> 	at org.apache.spark.scheduler.Task.run(Task.scala:121)
> 	at org.apache.spark.executor.Executor$TaskRunner$$anonfun$11.apply(Executor.scala:403)
> 	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1473)
> 	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:409)
> 	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)
> Found 1 deadlock.
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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