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

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

Venkata krishnan Sowrirajan created SPARK-27338:
---------------------------------------------------

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


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