Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-27338

Deadlock between TaskMemoryManager and UnsafeExternalSorter$SpillableIterator

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.4.0
    • 2.3.4, 2.4.2, 3.0.0
    • Spark Core
    • None

    Description

      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:

      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.
      

      Attachments

        Activity

          People

            vsowrirajan Venkata krishnan Sowrirajan
            vsowrirajan Venkata krishnan Sowrirajan
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: