Details
-
Bug
-
Status: Resolved
-
Major
-
Resolution: Duplicate
-
1.6.1
-
None
-
None
-
None
-
spark.memory.useLegacyMode=true
Description
the condition causes dead lock is:
Thread 1. 'BlockManagerSlaveEndpoint' receives 'RemoveBroadcast' or 'RemoveBlock' or .., executor begins to remove block named 'blockId1'
Thread 2. the other rdd begins to run and must 'evictBlocksToFreeSpace()' for more memory, unfortunately the chosen block to evict is 'blockId1'
As follows,
thread 1 is holding blockId1's blockInfo lock(0x000000053ab39f58) and waitting for StaticMemoryManager's lock(0x000000039f04fea8).
thread 2 is holding StaticMemoryManager's lock(0x000000039f04fea8) and waitting for blockId1's blockInfo lock(0x000000053ab39f58).
This condition causes dead lock.
stackTrace:
Found one Java-level deadlock:
=============================
"block-manager-slave-async-thread-pool-24":
waiting to lock monitor 0x00007f0b004ec278 (object 0x000000039f04fea8, a org.apache.spark.memory.StaticMemoryManager),
which is held by "Executor task launch worker-11"
"Executor task launch worker-11":
waiting to lock monitor 0x00007f0b01354018 (object 0x000000053ab39f58, a org.apache.spark.storage.BlockInfo),
which is held by "block-manager-slave-async-thread-pool-22"
"block-manager-slave-async-thread-pool-22":
waiting to lock monitor 0x00007f0b004ec278 (object 0x000000039f04fea8, a org.apache.spark.memory.StaticMemoryManager),
which is held by "Executor task launch worker-11"
Java stack information for the threads listed above:
===================================================
"Executor task launch worker-11":
at org.apache.spark.storage.BlockManager.dropFromMemory(BlockManager.scala:1032)
- waiting to lock <0x000000053ab39f58> (a org.apache.spark.storage.BlockInfo)
at org.apache.spark.storage.BlockManager.dropFromMemory(BlockManager.scala:1009)
at org.apache.spark.storage.MemoryStore$$anonfun$evictBlocksToFreeSpace$2.apply(MemoryStore.scala:460)
at org.apache.spark.storage.MemoryStore$$anonfun$evictBlocksToFreeSpace$2.apply(MemoryStore.scala:449)
at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
at org.apache.spark.storage.MemoryStore.evictBlocksToFreeSpace(MemoryStore.scala:449) - locked <0x000000039f04fea8> (a org.apache.spark.memory.StaticMemoryManager)
at org.apache.spark.memory.StorageMemoryPool.acquireMemory(StorageMemoryPool.scala:89) - locked <0x000000039f04fea8> (a org.apache.spark.memory.StaticMemoryManager)
at org.apache.spark.memory.StaticMemoryManager.acquireUnrollMemory(StaticMemoryManager.scala:83) - locked <0x000000039f04fea8> (a org.apache.spark.memory.StaticMemoryManager)
at org.apache.spark.storage.MemoryStore.reserveUnrollMemoryForThisTask(MemoryStore.scala:493) - locked <0x000000039f04fea8> (a org.apache.spark.memory.StaticMemoryManager)
at org.apache.spark.storage.MemoryStore.unrollSafely(MemoryStore.scala:291)
at org.apache.spark.CacheManager.putInBlockManager(CacheManager.scala:178)
at org.apache.spark.CacheManager.getOrCompute(CacheManager.scala:85)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:268)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
at org.apache.spark.scheduler.Task.run(Task.scala:89)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:744)
"block-manager-slave-async-thread-pool-22":
at org.apache.spark.storage.MemoryStore.remove(MemoryStore.scala:216) - waiting to lock <0x000000039f04fea8> (a org.apache.spark.memory.StaticMemoryManager)
at org.apache.spark.storage.BlockManager.removeBlock(BlockManager.scala:1114) - locked <0x000000053ab39f58> (a org.apache.spark.storage.BlockInfo)
at org.apache.spark.storage.BlockManager$$anonfun$removeBroadcast$2.apply(BlockManager.scala:1101)
at org.apache.spark.storage.BlockManager$$anonfun$removeBroadcast$2.apply(BlockManager.scala:1101)
at scala.collection.immutable.Set$Set2.foreach(Set.scala:94)
at org.apache.spark.storage.BlockManager.removeBroadcast(BlockManager.scala:1101)
at org.apache.spark.storage.BlockManagerSlaveEndpoint$$anonfun$receiveAndReply$1$$anonfun$applyOrElse$4.apply$mcI$sp(BlockManagerSlaveEndpoint.scala:65)
at org.apache.spark.storage.BlockManagerSlaveEndpoint$$anonfun$receiveAndReply$1$$anonfun$applyOrElse$4.apply(BlockManagerSlaveEndpoint.scala:65)
at org.apache.spark.storage.BlockManagerSlaveEndpoint$$anonfun$receiveAndReply$1$$anonfun$applyOrElse$4.apply(BlockManagerSlaveEndpoint.scala:65)
at org.apache.spark.storage.BlockManagerSlaveEndpoint$$anonfun$1.apply(BlockManagerSlaveEndpoint.scala:81)
at scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24)
at scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:744)
Found 1 deadlock.
Attachments
Issue Links
- duplicates
-
SPARK-13566 Deadlock between MemoryStore and BlockManager
- Resolved