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

qian han updated SPARK-26265:
-----------------------------
    Description: 
The application is running on a cluster with 72000 core and 182000G mem.

 

call stack is as follow:

org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:364)
 org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:422) 
org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:357) 
org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:193)
 
org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)
 sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
 java.lang.reflect.Method.invoke(Method.java:498) 
org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52) 
org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:894)
 org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:198) 
org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:228) 
org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:137) 
org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)

 

jstack information is as follow:

Found one Java-level deadlock: ============================= 
"Thread-ScriptTransformation-Feed": waiting to lock monitor 0x0000000000e0cb18 
(object 0x00000002f1641538, a org.apache.spark.memory.TaskMemoryManager), which 
is held by "Executor task launch worker for task 18899" "Executor task launch 
worker for task 18899": waiting to lock monitor 0x0000000000e09788 (object 
0x0000000302faa3b0, a org.apache.spark.unsafe.map.BytesToBytesMap$MapIterator), 
which is held by "Thread-ScriptTransformation-Feed" Java stack information for 
the threads listed above: =================================================== 
"Thread-ScriptTransformation-Feed": at 
org.apache.spark.memory.TaskMemoryManager.freePage(TaskMemoryManager.java:332) 
- waiting to lock <0x00000002f1641538> (a 
org.apache.spark.memory.TaskMemoryManager) at 
org.apache.spark.memory.MemoryConsumer.freePage(MemoryConsumer.java:130) at 
org.apache.spark.unsafe.map.BytesToBytesMap.access$300(BytesToBytesMap.java:66) 
at 
org.apache.spark.unsafe.map.BytesToBytesMap$MapIterator.advanceToNextPage(BytesToBytesMap.java:274)
 - locked <0x0000000302faa3b0> (a 
org.apache.spark.unsafe.map.BytesToBytesMap$MapIterator) at 
org.apache.spark.unsafe.map.BytesToBytesMap$MapIterator.next(BytesToBytesMap.java:313)
 at 
org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap$1.next(UnsafeFixedWidthAggregationMap.java:173)
 at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage2.processNext(Unknown
 Source) at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
 at 
org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$10$$anon$1.hasNext(WholeStageCodegenExec.scala:614)
 at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408) at 
scala.collection.Iterator$class.foreach(Iterator.scala:893) at 
scala.collection.AbstractIterator.foreach(Iterator.scala:1336) at 
org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread$$anonfun$run$1.apply$mcV$sp(ScriptTransformationExec.scala:281)
 at 
org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread$$anonfun$run$1.apply(ScriptTransformationExec.scala:270)
 at 
org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread$$anonfun$run$1.apply(ScriptTransformationExec.scala:270)
 at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1995) at 
org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread.run(ScriptTransformationExec.scala:270)
 "Executor task launch worker for task 18899": at 
org.apache.spark.unsafe.map.BytesToBytesMap$MapIterator.spill(BytesToBytesMap.java:345)
 - waiting to lock <0x0000000302faa3b0> (a 
org.apache.spark.unsafe.map.BytesToBytesMap$MapIterator) at 
org.apache.spark.unsafe.map.BytesToBytesMap.spill(BytesToBytesMap.java:772) at 
org.apache.spark.memory.TaskMemoryManager.acquireExecutionMemory(TaskMemoryManager.java:180)
 - locked <0x00000002f1641538> (a org.apache.spark.memory.TaskMemoryManager) at 
org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:283)
 at 
org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:117) at 
org.apache.spark.shuffle.sort.ShuffleExternalSorter.acquireNewPageIfNecessary(ShuffleExternalSorter.java:371)
 at 
org.apache.spark.shuffle.sort.ShuffleExternalSorter.insertRecord(ShuffleExternalSorter.java:394)
 at 
org.apache.spark.shuffle.sort.UnsafeShuffleWriter.insertRecordIntoSorter(UnsafeShuffleWriter.java:267)
 at 
org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:188)
 at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96) 
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53) 
at org.apache.spark.scheduler.Task.run(Task.scala:109) at 
org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345) 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:748) Found 1 deadlock.

  was:2018-12-03 10:29:25 Full thread dump Java HotSpot(TM) 64-Bit Server VM 
(25.131-b11 mixed mode): "Attach Listener" #147 daemon prio=9 os_prio=0 
tid=0x00007f4df6eb1000 nid=0x320f83 waiting on condition [0x0000000000000000] 
java.lang.Thread.State: RUNNABLE "Thread-ScriptTransformation-Feed" #126 daemon 
prio=5 os_prio=0 tid=0x00000000022cb800 nid=0x1f1574 waiting for monitor entry 
[0x00007f4d9fecf000] java.lang.Thread.State: BLOCKED (on object monitor) at 
org.apache.spark.memory.TaskMemoryManager.freePage(TaskMemoryManager.java:332) 
- waiting to lock <0x00000002f1641538> (a 
org.apache.spark.memory.TaskMemoryManager) at 
org.apache.spark.memory.MemoryConsumer.freePage(MemoryConsumer.java:130) at 
org.apache.spark.unsafe.map.BytesToBytesMap.access$300(BytesToBytesMap.java:66) 
at 
org.apache.spark.unsafe.map.BytesToBytesMap$MapIterator.advanceToNextPage(BytesToBytesMap.java:274)
 - locked <0x0000000302faa3b0> (a 
org.apache.spark.unsafe.map.BytesToBytesMap$MapIterator) at 
org.apache.spark.unsafe.map.BytesToBytesMap$MapIterator.next(BytesToBytesMap.java:313)
 at 
org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap$1.next(UnsafeFixedWidthAggregationMap.java:173)
 at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage2.processNext(Unknown
 Source) at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
 at 
org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$10$$anon$1.hasNext(WholeStageCodegenExec.scala:614)
 at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408) at 
scala.collection.Iterator$class.foreach(Iterator.scala:893) at 
scala.collection.AbstractIterator.foreach(Iterator.scala:1336) at 
org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread$$anonfun$run$1.apply$mcV$sp(ScriptTransformationExec.scala:281)
 at 
org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread$$anonfun$run$1.apply(ScriptTransformationExec.scala:270)
 at 
org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread$$anonfun$run$1.apply(ScriptTransformationExec.scala:270)
 at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1995) at 
org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread.run(ScriptTransformationExec.scala:270)
 "Thread-ScriptTransformation-STDERR-Consumer" #125 daemon prio=5 os_prio=0 
tid=0x0000000001032800 nid=0x1f1573 runnable [0x00007f4d9e8a7000] 
java.lang.Thread.State: RUNNABLE at java.io.FileInputStream.readBytes(Native 
Method) at java.io.FileInputStream.read(FileInputStream.java:255) at 
java.io.BufferedInputStream.fill(BufferedInputStream.java:246) at 
java.io.BufferedInputStream.read1(BufferedInputStream.java:286) at 
java.io.BufferedInputStream.read(BufferedInputStream.java:345) - locked 
<0x0000000302faaa60> (a java.lang.UNIXProcess$ProcessPipeInputStream) at 
java.io.FilterInputStream.read(FilterInputStream.java:107) at 
org.apache.spark.util.RedirectThread$$anonfun$run$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(Utils.scala:2935)
 at 
org.apache.spark.util.RedirectThread$$anonfun$run$1$$anonfun$apply$mcV$sp$1.apply(Utils.scala:2933)
 at 
org.apache.spark.util.RedirectThread$$anonfun$run$1$$anonfun$apply$mcV$sp$1.apply(Utils.scala:2933)
 at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1384) at 
org.apache.spark.util.RedirectThread$$anonfun$run$1.apply$mcV$sp(Utils.scala:2941)
 at org.apache.spark.util.RedirectThread$$anonfun$run$1.apply(Utils.scala:2941) 
at org.apache.spark.util.RedirectThread$$anonfun$run$1.apply(Utils.scala:2941) 
at scala.util.control.Exception$Catch.apply(Exception.scala:103) at 
org.apache.spark.util.RedirectThread.run(Utils.scala:2931) "process reaper" 
#124 daemon prio=10 os_prio=0 tid=0x00000000013e9800 nid=0x1f1572 runnable 
[0x00007f4da1f27000] java.lang.Thread.State: RUNNABLE at 
java.lang.UNIXProcess.waitForProcessExit(Native Method) at 
java.lang.UNIXProcess.lambda$initStreams$3(UNIXProcess.java:289) at 
java.lang.UNIXProcess$$Lambda$7/538185145.run(Unknown Source) 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:748) "shuffle-client-6-3" #106 daemon 
prio=5 os_prio=0 tid=0x0000000000ec3800 nid=0x1ed762 runnable 
[0x00007f4d9fdce000] java.lang.Thread.State: RUNNABLE at 
sun.nio.ch.EPollArrayWrapper.epollWait(Native Method) at 
sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269) at 
sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93) at 
sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86) - locked 
<0x00000002c6e1ed38> (a io.netty.channel.nio.SelectedSelectionKeySet) - locked 
<0x00000002c6efd450> (a java.util.Collections$UnmodifiableSet) - locked 
<0x00000002c6efd378> (a sun.nio.ch.EPollSelectorImpl) at 
sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97) at 
io.netty.channel.nio.SelectedSelectionKeySetSelector.select(SelectedSelectionKeySetSelector.java:62)
 at io.netty.channel.nio.NioEventLoop.select(NioEventLoop.java:753) at 
io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:409) at 
io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858)
 at 
io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138)
 at java.lang.Thread.run(Thread.java:748) "shuffle-client-6-4" #105 daemon 
prio=5 os_prio=0 tid=0x0000000001347000 nid=0x1ed761 runnable 
[0x00007f4d9fccd000] java.lang.Thread.State: RUNNABLE at 
sun.nio.ch.EPollArrayWrapper.epollWait(Native Method) at 
sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269) at 
sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93) at 
sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86) - locked 
<0x00000002c6b41160> (a io.netty.channel.nio.SelectedSelectionKeySet) - locked 
<0x00000002c6b42260> (a java.util.Collections$UnmodifiableSet) - locked 
<0x00000002c6b42188> (a sun.nio.ch.EPollSelectorImpl) at 
sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97) at 
io.netty.channel.nio.SelectedSelectionKeySetSelector.select(SelectedSelectionKeySetSelector.java:62)
 at io.netty.channel.nio.NioEventLoop.select(NioEventLoop.java:753) at 
io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:409) at 
io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858)
 at 
io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138)
 at java.lang.Thread.run(Thread.java:748) "shuffle-client-6-2" #104 daemon 
prio=5 os_prio=0 tid=0x00000000010b3000 nid=0x1ed760 runnable 
[0x00007f4d9fbcc000] java.lang.Thread.State: RUNNABLE at 
sun.nio.ch.EPollArrayWrapper.epollWait(Native Method) at 
sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269) at 
sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93) at 
sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86) - locked 
<0x00000002c6b7ee80> (a io.netty.channel.nio.SelectedSelectionKeySet) - locked 
<0x00000002c6b7ff80> (a java.util.Collections$UnmodifiableSet) - locked 
<0x00000002c6b7fea8> (a sun.nio.ch.EPollSelectorImpl) at 
sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97) at 
io.netty.channel.nio.SelectedSelectionKeySetSelector.select(SelectedSelectionKeySetSelector.java:62)
 at io.netty.channel.nio.NioEventLoop.select(NioEventLoop.java:753) at 
io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:409) at 
io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858)
 at 
io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138)
 at java.lang.Thread.run(Thread.java:748) "shuffle-client-4-1" #103 daemon 
prio=5 os_prio=0 tid=0x0000000000759800 nid=0x1ed746 runnable 
[0x00007f4da15e6000] java.lang.Thread.State: RUNNABLE at 
sun.nio.ch.EPollArrayWrapper.epollWait(Native Method) at 
sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269) at 
sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93) at 
sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86) - locked 
<0x00000002c6bc5920> (a io.netty.channel.nio.SelectedSelectionKeySet) - locked 
<0x00000002c6bc6988> (a java.util.Collections$UnmodifiableSet) - locked 
<0x00000002c6bc5888> (a sun.nio.ch.EPollSelectorImpl) at 
sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97) at 
io.netty.channel.nio.SelectedSelectionKeySetSelector.select(SelectedSelectionKeySetSelector.java:62)
 at io.netty.channel.nio.NioEventLoop.select(NioEventLoop.java:753) at 
io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:409) at 
io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858)
 at 
io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138)
 at java.lang.Thread.run(Thread.java:748) "files-client-7-1" #101 daemon prio=5 
os_prio=0 tid=0x000000000158d000 nid=0x1ed73d runnable [0x00007f4d9f5c6000] 
java.lang.Thread.State: RUNNABLE at 
sun.nio.ch.EPollArrayWrapper.epollWait(Native Method) at 
sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269) at 
sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93) at 
sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86) - locked 
<0x00000002c6c11a98> (a io.netty.channel.nio.SelectedSelectionKeySet) - locked 
<0x00000002c6c12b70> (a java.util.Collections$UnmodifiableSet) - locked 
<0x00000002c6c119f0> (a sun.nio.ch.EPollSelectorImpl) at 
sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97) at 
io.netty.channel.nio.SelectedSelectionKeySetSelector.select(SelectedSelectionKeySetSelector.java:62)
 at io.netty.channel.nio.NioEventLoop.select(NioEventLoop.java:753) at 
io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:409) at 
io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858)
 at 
io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138)
 at java.lang.Thread.run(Thread.java:748) "Executor task launch worker for task 
18899" #99 daemon prio=5 os_prio=0 tid=0x00007f4dec30b800 nid=0x1ed73b waiting 
for monitor entry [0x00007f4d9f2c2000] java.lang.Thread.State: BLOCKED (on 
object monitor) at 
org.apache.spark.unsafe.map.BytesToBytesMap$MapIterator.spill(BytesToBytesMap.java:345)
 - waiting to lock <0x0000000302faa3b0> (a 
org.apache.spark.unsafe.map.BytesToBytesMap$MapIterator) at 
org.apache.spark.unsafe.map.BytesToBytesMap.spill(BytesToBytesMap.java:772) at 
org.apache.spark.memory.TaskMemoryManager.acquireExecutionMemory(TaskMemoryManager.java:180)
 - locked <0x00000002f1641538> (a org.apache.spark.memory.TaskMemoryManager) at 
org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:283)
 at 
org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:117) at 
org.apache.spark.shuffle.sort.ShuffleExternalSorter.acquireNewPageIfNecessary(ShuffleExternalSorter.java:371)
 at 
org.apache.spark.shuffle.sort.ShuffleExternalSorter.insertRecord(ShuffleExternalSorter.java:394)
 at 
org.apache.spark.shuffle.sort.UnsafeShuffleWriter.insertRecordIntoSorter(UnsafeShuffleWriter.java:267)
 at 
org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:188)
 at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96) 
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53) 
at org.apache.spark.scheduler.Task.run(Task.scala:109) at 
org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345) 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:748) "driver-heartbeater" #96 daemon prio=5 
os_prio=0 tid=0x00007f4dec195000 nid=0x1ed733 waiting on condition 
[0x00007f4d9f3c4000] java.lang.Thread.State: TIMED_WAITING (parking) at 
sun.misc.Unsafe.park(Native Method) - parking to wait for <0x00000002c6b681d8> 
(a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) at 
java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215) at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
 at 
java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1093)
 at 
java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:809)
 at 
java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1067) 
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1127) 
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) 
at java.lang.Thread.run(Thread.java:748) "shuffle-client-6-1" #95 daemon prio=5 
os_prio=0 tid=0x00007f4dec386800 nid=0x1ed732 runnable [0x00007f4d9f4c5000] 
java.lang.Thread.State: RUNNABLE at 
sun.nio.ch.EPollArrayWrapper.epollWait(Native Method) at 
sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269) at 
sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93) at 
sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86) - locked 
<0x00000002c6b684d0> (a io.netty.channel.nio.SelectedSelectionKeySet) - locked 
<0x00000002c6b684e8> (a java.util.Collections$UnmodifiableSet) - locked 
<0x00000002c6b68488> (a sun.nio.ch.EPollSelectorImpl) at 
sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97) at 
io.netty.channel.nio.SelectedSelectionKeySetSelector.select(SelectedSelectionKeySetSelector.java:62)
 at io.netty.channel.nio.NioEventLoop.select(NioEventLoop.java:753) at 
io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:409) at 
io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858)
 at 
io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138)
 at java.lang.Thread.run(Thread.java:748) "shuffle-server-5-1" #93 daemon 
prio=5 os_prio=0 tid=0x00007f4dec7a0000 nid=0x1ed727 runnable 
[0x00007f4d9facb000] java.lang.Thread.State: RUNNABLE at 
sun.nio.ch.EPollArrayWrapper.epollWait(Native Method) at 
sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269) at 
sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93) at 
sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86) - locked 
<0x00000002c6b687d8> (a io.netty.channel.nio.SelectedSelectionKeySet) - locked 
<0x00000002c6b687f0> (a java.util.Collections$UnmodifiableSet) - locked 
<0x00000002c6b68790> (a sun.nio.ch.EPollSelectorImpl) at 
sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97) at 
io.netty.channel.nio.SelectedSelectionKeySetSelector.select(SelectedSelectionKeySetSelector.java:62)
 at io.netty.channel.nio.NioEventLoop.select(NioEventLoop.java:753) at 
io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:409) at 
io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858)
 at 
io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138)
 at java.lang.Thread.run(Thread.java:748) 
"metrics-bytedance-reporter-1-thread-1" #92 daemon prio=5 os_prio=0 
tid=0x00007f4e0a673000 nid=0x1ed71e waiting on condition [0x00007f4d9f9ca000] 
java.lang.Thread.State: TIMED_WAITING (parking) at sun.misc.Unsafe.park(Native 
Method) - parking to wait for <0x00000002c6b68a48> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) at 
java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215) at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
 at 
java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1093)
 at 
java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:809)
 at 
java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1067) 
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1127) 
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) 
at java.lang.Thread.run(Thread.java:748) "RemoteBlock-temp-file-clean-thread" 
#91 daemon prio=5 os_prio=0 tid=0x00007f4e0a54e800 nid=0x1ed70a in 
Object.wait() [0x00007f4d9f8c9000] java.lang.Thread.State: TIMED_WAITING (on 
object monitor) at java.lang.Object.wait(Native Method) at 
java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:143) - locked 
<0x00000002c6b68cb0> (a java.lang.ref.ReferenceQueue$Lock) at 
org.apache.spark.storage.BlockManager$RemoteBlockTempFileManager.org$apache$spark$storage$BlockManager$RemoteBlockTempFileManager$$keepCleaning(BlockManager.scala:1630)
 at 
org.apache.spark.storage.BlockManager$RemoteBlockTempFileManager$$anon$1.run(BlockManager.scala:1608)
 "rpc-client-3-1" #89 daemon prio=5 os_prio=0 tid=0x00007f4e0a2b7000 
nid=0x1ed6f6 runnable [0x00007f4da03d4000] java.lang.Thread.State: RUNNABLE at 
sun.nio.ch.EPollArrayWrapper.epollWait(Native Method) at 
sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269) at 
sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93) at 
sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86) - locked 
<0x00000002c6b68f18> (a io.netty.channel.nio.SelectedSelectionKeySet) - locked 
<0x00000002c6b68f30> (a java.util.Collections$UnmodifiableSet) - locked 
<0x00000002c6b68ed0> (a sun.nio.ch.EPollSelectorImpl) at 
sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97) at 
io.netty.channel.nio.SelectedSelectionKeySetSelector.select(SelectedSelectionKeySetSelector.java:62)
 at io.netty.channel.nio.NioEventLoop.select(NioEventLoop.java:753) at 
io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:409) at 
io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858)
 at 
io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138)
 at java.lang.Thread.run(Thread.java:748) "netty-rpc-env-timeout" #88 daemon 
prio=5 os_prio=0 tid=0x00007f4e0a2a9000 nid=0x1ed6f5 waiting on condition 
[0x00007f4d9f7c8000] java.lang.Thread.State: TIMED_WAITING (parking) at 
sun.misc.Unsafe.park(Native Method) - parking to wait for <0x00000002c6b69188> 
(a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) at 
java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215) at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
 at 
java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1093)
 at 
java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:809)
 at 
java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1067) 
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1127) 
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) 
at java.lang.Thread.run(Thread.java:748) "dispatcher-event-loop-3" #86 daemon 
prio=5 os_prio=0 tid=0x00007f4e0a2bc800 nid=0x1ed6e5 waiting on condition 
[0x00007f4d9efc0000] java.lang.Thread.State: WAITING (parking) at 
sun.misc.Unsafe.park(Native Method) - parking to wait for <0x00000002c6b69640> 
(a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) at 
java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
 at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442) 
at org.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:215) 
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:748) "dispatcher-event-loop-2" #85 daemon 
prio=5 os_prio=0 tid=0x00007f4e0a2d1800 nid=0x1ed6e4 waiting on condition 
[0x00007f4d9edbe000] java.lang.Thread.State: WAITING (parking) at 
sun.misc.Unsafe.park(Native Method) - parking to wait for <0x00000002c6b69640> 
(a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) at 
java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
 at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442) 
at org.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:215) 
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:748) "dispatcher-event-loop-1" #84 daemon 
prio=5 os_prio=0 tid=0x00007f4e0a29c800 nid=0x1ed6e3 waiting on condition 
[0x00007f4d9eebf000] java.lang.Thread.State: WAITING (parking) at 
sun.misc.Unsafe.park(Native Method) - parking to wait for <0x00000002c6b69640> 
(a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) at 
java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
 at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442) 
at org.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:215) 
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:748) "dispatcher-event-loop-0" #83 daemon 
prio=5 os_prio=0 tid=0x00007f4e0a2d3800 nid=0x1ed6e2 waiting on condition 
[0x00007f4d9f0c1000] java.lang.Thread.State: WAITING (parking) at 
sun.misc.Unsafe.park(Native Method) - parking to wait for <0x00000002c6b69640> 
(a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) at 
java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
 at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442) 
at org.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:215) 
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:748) "Service Thread" #21 daemon prio=9 
os_prio=0 tid=0x00007f4e08692800 nid=0x1ed51b runnable [0x0000000000000000] 
java.lang.Thread.State: RUNNABLE "C1 CompilerThread14" #20 daemon prio=9 
os_prio=0 tid=0x00007f4e0868f800 nid=0x1ed510 waiting on condition 
[0x0000000000000000] java.lang.Thread.State: RUNNABLE "C1 CompilerThread13" #19 
daemon prio=9 os_prio=0 tid=0x00007f4e0868d800 nid=0x1ed50d waiting on 
condition [0x0000000000000000] java.lang.Thread.State: RUNNABLE "C1 
CompilerThread12" #18 daemon prio=9 os_prio=0 tid=0x00007f4e0868c000 
nid=0x1ed50c waiting on condition [0x0000000000000000] java.lang.Thread.State: 
RUNNABLE "C1 CompilerThread11" #17 daemon prio=9 os_prio=0 
tid=0x00007f4e08681000 nid=0x1ed50a waiting on condition [0x0000000000000000] 
java.lang.Thread.State: RUNNABLE "C1 CompilerThread10" #16 daemon prio=9 
os_prio=0 tid=0x00007f4e0867f000 nid=0x1ed509 waiting on condition 
[0x0000000000000000] java.lang.Thread.State: RUNNABLE "C2 CompilerThread9" #15 
daemon prio=9 os_prio=0 tid=0x00007f4e08674800 nid=0x1ed508 waiting on 
condition [0x0000000000000000] java.lang.Thread.State: RUNNABLE "C2 
CompilerThread8" #14 daemon prio=9 os_prio=0 tid=0x00007f4e08671800 
nid=0x1ed507 waiting on condition [0x0000000000000000] java.lang.Thread.State: 
RUNNABLE "C2 CompilerThread7" #13 daemon prio=9 os_prio=0 
tid=0x00007f4e08667800 nid=0x1ed506 waiting on condition [0x0000000000000000] 
java.lang.Thread.State: RUNNABLE "C2 CompilerThread6" #12 daemon prio=9 
os_prio=0 tid=0x00007f4e0865d800 nid=0x1ed505 waiting on condition 
[0x0000000000000000] java.lang.Thread.State: RUNNABLE "C2 CompilerThread5" #11 
daemon prio=9 os_prio=0 tid=0x00007f4e08640800 nid=0x1ed504 waiting on 
condition [0x0000000000000000] java.lang.Thread.State: RUNNABLE "C2 
CompilerThread4" #10 daemon prio=9 os_prio=0 tid=0x00007f4e0862e800 
nid=0x1ed503 waiting on condition [0x0000000000000000] java.lang.Thread.State: 
RUNNABLE "C2 CompilerThread3" #9 daemon prio=9 os_prio=0 tid=0x00007f4e08613000 
nid=0x1ed502 waiting on condition [0x0000000000000000] java.lang.Thread.State: 
RUNNABLE "C2 CompilerThread2" #8 daemon prio=9 os_prio=0 tid=0x00007f4e08610800 
nid=0x1ed501 waiting on condition [0x0000000000000000] java.lang.Thread.State: 
RUNNABLE "C2 CompilerThread1" #7 daemon prio=9 os_prio=0 tid=0x00007f4e0860e000 
nid=0x1ed500 waiting on condition [0x0000000000000000] java.lang.Thread.State: 
RUNNABLE "C2 CompilerThread0" #6 daemon prio=9 os_prio=0 tid=0x00007f4e0860c000 
nid=0x1ed4ff waiting on condition [0x0000000000000000] java.lang.Thread.State: 
RUNNABLE "Signal Dispatcher" #5 daemon prio=9 os_prio=0 tid=0x00007f4e08609800 
nid=0x1ed4fe runnable [0x0000000000000000] java.lang.Thread.State: RUNNABLE 
"Surrogate Locker Thread (Concurrent GC)" #4 daemon prio=9 os_prio=0 
tid=0x00007f4e08608800 nid=0x1ed4fd waiting on condition [0x0000000000000000] 
java.lang.Thread.State: RUNNABLE "Finalizer" #3 daemon prio=8 os_prio=0 
tid=0x00007f4e085d5800 nid=0x1ed4f1 in Object.wait() [0x00007f4da8e31000] 
java.lang.Thread.State: WAITING (on object monitor) at 
java.lang.Object.wait(Native Method) at 
java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:143) - locked 
<0x00000002c6513c30> (a java.lang.ref.ReferenceQueue$Lock) at 
java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:164) at 
java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:209) "Reference 
Handler" #2 daemon prio=10 os_prio=0 tid=0x00007f4e085cb000 nid=0x1ed4f0 in 
Object.wait() [0x00007f4da8f32000] java.lang.Thread.State: WAITING (on object 
monitor) at java.lang.Object.wait(Native Method) at 
java.lang.Object.wait(Object.java:502) at 
java.lang.ref.Reference.tryHandlePending(Reference.java:191) - locked 
<0x00000002c6513c70> (a java.lang.ref.Reference$Lock) at 
java.lang.ref.Reference$ReferenceHandler.run(Reference.java:153) "main" #1 
prio=5 os_prio=0 tid=0x00007f4e08027000 nid=0x1ed440 waiting on condition 
[0x00007f4e0f198000] java.lang.Thread.State: TIMED_WAITING (parking) at 
sun.misc.Unsafe.park(Native Method) - parking to wait for <0x00000002c6b69d28> 
(a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) at 
java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215) at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
 at 
java.util.concurrent.ThreadPoolExecutor.awaitTermination(ThreadPoolExecutor.java:1465)
 at 
org.apache.spark.rpc.netty.Dispatcher.awaitTermination(Dispatcher.scala:186) at 
org.apache.spark.rpc.netty.NettyRpcEnv.awaitTermination(NettyRpcEnv.scala:285) 
at 
org.apache.spark.executor.CoarseGrainedExecutorBackend$$anonfun$run$1.apply$mcV$sp(CoarseGrainedExecutorBackend.scala:236)
 at 
org.apache.spark.deploy.SparkHadoopUtil$$anon$2.run(SparkHadoopUtil.scala:65) 
at 
org.apache.spark.deploy.SparkHadoopUtil$$anon$2.run(SparkHadoopUtil.scala:64) 
at java.security.AccessController.doPrivileged(Native Method) at 
javax.security.auth.Subject.doAs(Subject.java:422) at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1671)
 at 
org.apache.spark.deploy.SparkHadoopUtil.runAsSparkUser(SparkHadoopUtil.scala:64)
 at 
org.apache.spark.executor.CoarseGrainedExecutorBackend$.run(CoarseGrainedExecutorBackend.scala:188)
 at 
org.apache.spark.executor.CoarseGrainedExecutorBackend$.main(CoarseGrainedExecutorBackend.scala:293)
 at 
org.apache.spark.executor.CoarseGrainedExecutorBackend.main(CoarseGrainedExecutorBackend.scala)
 "VM Thread" os_prio=0 tid=0x00007f4e085c3800 nid=0x1ed4ed runnable "Gang 
worker#0 (Parallel GC Threads)" os_prio=0 tid=0x00007f4e08040800 nid=0x1ed442 
runnable "Gang worker#1 (Parallel GC Threads)" os_prio=0 tid=0x00007f4e08042000 
nid=0x1ed443 runnable "Gang worker#2 (Parallel GC Threads)" os_prio=0 
tid=0x00007f4e08044000 nid=0x1ed444 runnable "Gang worker#3 (Parallel GC 
Threads)" os_prio=0 tid=0x00007f4e08045800 nid=0x1ed445 runnable "Gang worker#4 
(Parallel GC Threads)" os_prio=0 tid=0x00007f4e08047800 nid=0x1ed446 runnable 
"Gang worker#5 (Parallel GC Threads)" os_prio=0 tid=0x00007f4e08049000 
nid=0x1ed447 runnable "Gang worker#6 (Parallel GC Threads)" os_prio=0 
tid=0x00007f4e0804b000 nid=0x1ed448 runnable "Gang worker#7 (Parallel GC 
Threads)" os_prio=0 tid=0x00007f4e0804c800 nid=0x1ed449 runnable "Gang worker#8 
(Parallel GC Threads)" os_prio=0 tid=0x00007f4e0804e800 nid=0x1ed44a runnable 
"Gang worker#9 (Parallel GC Threads)" os_prio=0 tid=0x00007f4e08050000 
nid=0x1ed44b runnable "Gang worker#10 (Parallel GC Threads)" os_prio=0 
tid=0x00007f4e08052000 nid=0x1ed44c runnable "Gang worker#11 (Parallel GC 
Threads)" os_prio=0 tid=0x00007f4e08053800 nid=0x1ed44d runnable "Gang 
worker#12 (Parallel GC Threads)" os_prio=0 tid=0x00007f4e08055800 nid=0x1ed44e 
runnable "Gang worker#13 (Parallel GC Threads)" os_prio=0 
tid=0x00007f4e08057000 nid=0x1ed44f runnable "Gang worker#14 (Parallel GC 
Threads)" os_prio=0 tid=0x00007f4e08059000 nid=0x1ed450 runnable "Gang 
worker#15 (Parallel GC Threads)" os_prio=0 tid=0x00007f4e0805a800 nid=0x1ed451 
runnable "Gang worker#16 (Parallel GC Threads)" os_prio=0 
tid=0x00007f4e0805c800 nid=0x1ed452 runnable "Gang worker#17 (Parallel GC 
Threads)" os_prio=0 tid=0x00007f4e0805e000 nid=0x1ed453 runnable "Gang 
worker#18 (Parallel GC Threads)" os_prio=0 tid=0x00007f4e08060000 nid=0x1ed455 
runnable "Gang worker#19 (Parallel GC Threads)" os_prio=0 
tid=0x00007f4e08061800 nid=0x1ed456 runnable "Gang worker#20 (Parallel GC 
Threads)" os_prio=0 tid=0x00007f4e08063000 nid=0x1ed458 runnable "Gang 
worker#21 (Parallel GC Threads)" os_prio=0 tid=0x00007f4e08065000 nid=0x1ed45a 
runnable "Gang worker#22 (Parallel GC Threads)" os_prio=0 
tid=0x00007f4e08066800 nid=0x1ed45c runnable "Gang worker#23 (Parallel GC 
Threads)" os_prio=0 tid=0x00007f4e08068800 nid=0x1ed45e runnable "Gang 
worker#24 (Parallel GC Threads)" os_prio=0 tid=0x00007f4e0806a000 nid=0x1ed460 
runnable "Gang worker#25 (Parallel GC Threads)" os_prio=0 
tid=0x00007f4e0806c000 nid=0x1ed462 runnable "Gang worker#26 (Parallel GC 
Threads)" os_prio=0 tid=0x00007f4e0806d800 nid=0x1ed464 runnable "Gang 
worker#27 (Parallel GC Threads)" os_prio=0 tid=0x00007f4e0806f800 nid=0x1ed466 
runnable "Gang worker#28 (Parallel GC Threads)" os_prio=0 
tid=0x00007f4e08071000 nid=0x1ed468 runnable "Gang worker#29 (Parallel GC 
Threads)" os_prio=0 tid=0x00007f4e08073000 nid=0x1ed46a runnable "Gang 
worker#30 (Parallel GC Threads)" os_prio=0 tid=0x00007f4e08074800 nid=0x1ed46c 
runnable "Gang worker#31 (Parallel GC Threads)" os_prio=0 
tid=0x00007f4e08076800 nid=0x1ed46e runnable "Gang worker#32 (Parallel GC 
Threads)" os_prio=0 tid=0x00007f4e08078000 nid=0x1ed470 runnable "G1 Main 
Concurrent Mark GC Thread" os_prio=0 tid=0x00007f4e080cf800 nid=0x1ed4c8 
runnable "Gang worker#0 (G1 Parallel Marking Threads)" os_prio=0 
tid=0x00007f4e080d1000 nid=0x1ed4c9 runnable "Gang worker#1 (G1 Parallel 
Marking Threads)" os_prio=0 tid=0x00007f4e080d3000 nid=0x1ed4ca runnable "Gang 
worker#2 (G1 Parallel Marking Threads)" os_prio=0 tid=0x00007f4e080d4800 
nid=0x1ed4cb runnable "Gang worker#3 (G1 Parallel Marking Threads)" os_prio=0 
tid=0x00007f4e080d6800 nid=0x1ed4cc runnable "Gang worker#4 (G1 Parallel 
Marking Threads)" os_prio=0 tid=0x00007f4e080d8000 nid=0x1ed4cd runnable "Gang 
worker#5 (G1 Parallel Marking Threads)" os_prio=0 tid=0x00007f4e080da000 
nid=0x1ed4ce runnable "Gang worker#6 (G1 Parallel Marking Threads)" os_prio=0 
tid=0x00007f4e080db800 nid=0x1ed4cf runnable "Gang worker#7 (G1 Parallel 
Marking Threads)" os_prio=0 tid=0x00007f4e080dd800 nid=0x1ed4d0 runnable "G1 
Concurrent Refinement Thread#0" os_prio=0 tid=0x00007f4e080b7000 nid=0x1ed4b4 
runnable "G1 Concurrent Refinement Thread#1" os_prio=0 tid=0x00007f4e080b5000 
nid=0x1ed4b2 runnable "G1 Concurrent Refinement Thread#2" os_prio=0 
tid=0x00007f4e080b3800 nid=0x1ed4b0 runnable "G1 Concurrent Refinement 
Thread#3" os_prio=0 tid=0x00007f4e080b1800 nid=0x1ed4ae runnable "G1 Concurrent 
Refinement Thread#4" os_prio=0 tid=0x00007f4e080b0000 nid=0x1ed4ac runnable "G1 
Concurrent Refinement Thread#5" os_prio=0 tid=0x00007f4e080ae000 nid=0x1ed4aa 
runnable "G1 Concurrent Refinement Thread#6" os_prio=0 tid=0x00007f4e080ac000 
nid=0x1ed4a8 runnable "G1 Concurrent Refinement Thread#7" os_prio=0 
tid=0x00007f4e080aa800 nid=0x1ed4a6 runnable "G1 Concurrent Refinement 
Thread#8" os_prio=0 tid=0x00007f4e080a8800 nid=0x1ed4a4 runnable "G1 Concurrent 
Refinement Thread#9" os_prio=0 tid=0x00007f4e080a7000 nid=0x1ed4a2 runnable "G1 
Concurrent Refinement Thread#10" os_prio=0 tid=0x00007f4e080a5000 nid=0x1ed4a0 
runnable "G1 Concurrent Refinement Thread#11" os_prio=0 tid=0x00007f4e080a3800 
nid=0x1ed49e runnable "G1 Concurrent Refinement Thread#12" os_prio=0 
tid=0x00007f4e080a1800 nid=0x1ed49c runnable "G1 Concurrent Refinement 
Thread#13" os_prio=0 tid=0x00007f4e080a0000 nid=0x1ed49a runnable "G1 
Concurrent Refinement Thread#14" os_prio=0 tid=0x00007f4e0809e000 nid=0x1ed498 
runnable "G1 Concurrent Refinement Thread#15" os_prio=0 tid=0x00007f4e0809c800 
nid=0x1ed496 runnable "G1 Concurrent Refinement Thread#16" os_prio=0 
tid=0x00007f4e0809a800 nid=0x1ed495 runnable "G1 Concurrent Refinement 
Thread#17" os_prio=0 tid=0x00007f4e08098800 nid=0x1ed494 runnable "G1 
Concurrent Refinement Thread#18" os_prio=0 tid=0x00007f4e08097000 nid=0x1ed493 
runnable "G1 Concurrent Refinement Thread#19" os_prio=0 tid=0x00007f4e08095000 
nid=0x1ed491 runnable "G1 Concurrent Refinement Thread#20" os_prio=0 
tid=0x00007f4e08093800 nid=0x1ed48f runnable "G1 Concurrent Refinement 
Thread#21" os_prio=0 tid=0x00007f4e08091800 nid=0x1ed48d runnable "G1 
Concurrent Refinement Thread#22" os_prio=0 tid=0x00007f4e08090000 nid=0x1ed48b 
runnable "G1 Concurrent Refinement Thread#23" os_prio=0 tid=0x00007f4e0808e000 
nid=0x1ed489 runnable "G1 Concurrent Refinement Thread#24" os_prio=0 
tid=0x00007f4e0808c800 nid=0x1ed487 runnable "G1 Concurrent Refinement 
Thread#25" os_prio=0 tid=0x00007f4e0808a800 nid=0x1ed485 runnable "G1 
Concurrent Refinement Thread#26" os_prio=0 tid=0x00007f4e08089000 nid=0x1ed483 
runnable "G1 Concurrent Refinement Thread#27" os_prio=0 tid=0x00007f4e08087000 
nid=0x1ed481 runnable "G1 Concurrent Refinement Thread#28" os_prio=0 
tid=0x00007f4e08085000 nid=0x1ed47f runnable "G1 Concurrent Refinement 
Thread#29" os_prio=0 tid=0x00007f4e08083800 nid=0x1ed47d runnable "G1 
Concurrent Refinement Thread#30" os_prio=0 tid=0x00007f4e08081800 nid=0x1ed47b 
runnable "G1 Concurrent Refinement Thread#31" os_prio=0 tid=0x00007f4e08080000 
nid=0x1ed479 runnable "G1 Concurrent Refinement Thread#32" os_prio=0 
tid=0x00007f4e0807e000 nid=0x1ed477 runnable "G1 Concurrent Refinement 
Thread#33" os_prio=0 tid=0x00007f4e0807c800 nid=0x1ed475 runnable "VM Periodic 
Task Thread" os_prio=0 tid=0x00007f4e08695800 nid=0x1ed51d waiting on condition 
JNI global references: 330 Found one Java-level deadlock: 
============================= "Thread-ScriptTransformation-Feed": waiting to 
lock monitor 0x0000000000e0cb18 (object 0x00000002f1641538, a 
org.apache.spark.memory.TaskMemoryManager), which is held by "Executor task 
launch worker for task 18899" "Executor task launch worker for task 18899": 
waiting to lock monitor 0x0000000000e09788 (object 0x0000000302faa3b0, a 
org.apache.spark.unsafe.map.BytesToBytesMap$MapIterator), which is held by 
"Thread-ScriptTransformation-Feed" Java stack information for the threads 
listed above: =================================================== 
"Thread-ScriptTransformation-Feed": at 
org.apache.spark.memory.TaskMemoryManager.freePage(TaskMemoryManager.java:332) 
- waiting to lock <0x00000002f1641538> (a 
org.apache.spark.memory.TaskMemoryManager) at 
org.apache.spark.memory.MemoryConsumer.freePage(MemoryConsumer.java:130) at 
org.apache.spark.unsafe.map.BytesToBytesMap.access$300(BytesToBytesMap.java:66) 
at 
org.apache.spark.unsafe.map.BytesToBytesMap$MapIterator.advanceToNextPage(BytesToBytesMap.java:274)
 - locked <0x0000000302faa3b0> (a 
org.apache.spark.unsafe.map.BytesToBytesMap$MapIterator) at 
org.apache.spark.unsafe.map.BytesToBytesMap$MapIterator.next(BytesToBytesMap.java:313)
 at 
org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap$1.next(UnsafeFixedWidthAggregationMap.java:173)
 at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage2.processNext(Unknown
 Source) at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
 at 
org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$10$$anon$1.hasNext(WholeStageCodegenExec.scala:614)
 at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408) at 
scala.collection.Iterator$class.foreach(Iterator.scala:893) at 
scala.collection.AbstractIterator.foreach(Iterator.scala:1336) at 
org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread$$anonfun$run$1.apply$mcV$sp(ScriptTransformationExec.scala:281)
 at 
org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread$$anonfun$run$1.apply(ScriptTransformationExec.scala:270)
 at 
org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread$$anonfun$run$1.apply(ScriptTransformationExec.scala:270)
 at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1995) at 
org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread.run(ScriptTransformationExec.scala:270)
 "Executor task launch worker for task 18899": at 
org.apache.spark.unsafe.map.BytesToBytesMap$MapIterator.spill(BytesToBytesMap.java:345)
 - waiting to lock <0x0000000302faa3b0> (a 
org.apache.spark.unsafe.map.BytesToBytesMap$MapIterator) at 
org.apache.spark.unsafe.map.BytesToBytesMap.spill(BytesToBytesMap.java:772) at 
org.apache.spark.memory.TaskMemoryManager.acquireExecutionMemory(TaskMemoryManager.java:180)
 - locked <0x00000002f1641538> (a org.apache.spark.memory.TaskMemoryManager) at 
org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:283)
 at 
org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:117) at 
org.apache.spark.shuffle.sort.ShuffleExternalSorter.acquireNewPageIfNecessary(ShuffleExternalSorter.java:371)
 at 
org.apache.spark.shuffle.sort.ShuffleExternalSorter.insertRecord(ShuffleExternalSorter.java:394)
 at 
org.apache.spark.shuffle.sort.UnsafeShuffleWriter.insertRecordIntoSorter(UnsafeShuffleWriter.java:267)
 at 
org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:188)
 at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96) 
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53) 
at org.apache.spark.scheduler.Task.run(Task.scala:109) at 
org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345) 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:748) Found 1 deadlock.


> deadlock between TaskMemoryManager and BytesToBytesMap$MapIterator
> ------------------------------------------------------------------
>
>                 Key: SPARK-26265
>                 URL: https://issues.apache.org/jira/browse/SPARK-26265
>             Project: Spark
>          Issue Type: Bug
>          Components: Spark Core
>    Affects Versions: 2.3.2
>            Reporter: qian han
>            Priority: Major
>
> The application is running on a cluster with 72000 core and 182000G mem.
>  
> call stack is as follow:
> org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:364)
>  org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:422) 
> org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:357) 
> org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:193)
>  
> org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)
>  sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>  java.lang.reflect.Method.invoke(Method.java:498) 
> org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52) 
> org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:894)
>  org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:198) 
> org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:228) 
> org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:137) 
> org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
>  
> jstack information is as follow:
> Found one Java-level deadlock: ============================= 
> "Thread-ScriptTransformation-Feed": waiting to lock monitor 
> 0x0000000000e0cb18 (object 0x00000002f1641538, a 
> org.apache.spark.memory.TaskMemoryManager), which is held by "Executor task 
> launch worker for task 18899" "Executor task launch worker for task 18899": 
> waiting to lock monitor 0x0000000000e09788 (object 0x0000000302faa3b0, a 
> org.apache.spark.unsafe.map.BytesToBytesMap$MapIterator), which is held by 
> "Thread-ScriptTransformation-Feed" Java stack information for the threads 
> listed above: =================================================== 
> "Thread-ScriptTransformation-Feed": at 
> org.apache.spark.memory.TaskMemoryManager.freePage(TaskMemoryManager.java:332)
>  - waiting to lock <0x00000002f1641538> (a 
> org.apache.spark.memory.TaskMemoryManager) at 
> org.apache.spark.memory.MemoryConsumer.freePage(MemoryConsumer.java:130) at 
> org.apache.spark.unsafe.map.BytesToBytesMap.access$300(BytesToBytesMap.java:66)
>  at 
> org.apache.spark.unsafe.map.BytesToBytesMap$MapIterator.advanceToNextPage(BytesToBytesMap.java:274)
>  - locked <0x0000000302faa3b0> (a 
> org.apache.spark.unsafe.map.BytesToBytesMap$MapIterator) at 
> org.apache.spark.unsafe.map.BytesToBytesMap$MapIterator.next(BytesToBytesMap.java:313)
>  at 
> org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap$1.next(UnsafeFixedWidthAggregationMap.java:173)
>  at 
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage2.processNext(Unknown
>  Source) at 
> org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
>  at 
> org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$10$$anon$1.hasNext(WholeStageCodegenExec.scala:614)
>  at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408) at 
> scala.collection.Iterator$class.foreach(Iterator.scala:893) at 
> scala.collection.AbstractIterator.foreach(Iterator.scala:1336) at 
> org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread$$anonfun$run$1.apply$mcV$sp(ScriptTransformationExec.scala:281)
>  at 
> org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread$$anonfun$run$1.apply(ScriptTransformationExec.scala:270)
>  at 
> org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread$$anonfun$run$1.apply(ScriptTransformationExec.scala:270)
>  at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1995) at 
> org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread.run(ScriptTransformationExec.scala:270)
>  "Executor task launch worker for task 18899": at 
> org.apache.spark.unsafe.map.BytesToBytesMap$MapIterator.spill(BytesToBytesMap.java:345)
>  - waiting to lock <0x0000000302faa3b0> (a 
> org.apache.spark.unsafe.map.BytesToBytesMap$MapIterator) at 
> org.apache.spark.unsafe.map.BytesToBytesMap.spill(BytesToBytesMap.java:772) 
> at 
> org.apache.spark.memory.TaskMemoryManager.acquireExecutionMemory(TaskMemoryManager.java:180)
>  - locked <0x00000002f1641538> (a org.apache.spark.memory.TaskMemoryManager) 
> at 
> org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:283)
>  at 
> org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:117) 
> at 
> org.apache.spark.shuffle.sort.ShuffleExternalSorter.acquireNewPageIfNecessary(ShuffleExternalSorter.java:371)
>  at 
> org.apache.spark.shuffle.sort.ShuffleExternalSorter.insertRecord(ShuffleExternalSorter.java:394)
>  at 
> org.apache.spark.shuffle.sort.UnsafeShuffleWriter.insertRecordIntoSorter(UnsafeShuffleWriter.java:267)
>  at 
> org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:188)
>  at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96) at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53) at 
> org.apache.spark.scheduler.Task.run(Task.scala:109) at 
> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345) 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:748) Found 1 deadlock.



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

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to