[
https://issues.apache.org/jira/browse/SPARK-26265?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16716139#comment-16716139
]
ASF GitHub Bot commented on SPARK-26265:
----------------------------------------
viirya commented on issue #23272: [SPARK-26265][Core] Fix deadlock in
BytesToBytesMap.MapIterator when locking both BytesToBytesMap.MapIterator and
TaskMemoryManager
URL: https://github.com/apache/spark/pull/23272#issuecomment-446067447
Oh, you meant that the page is freed by other using this map or iterator.
Is it a problem?
I think it should not be a case that more than one consumers free the same
page at the same time.
On Tue, Dec 11, 2018, 11:34 Wenchen Fan <[email protected] wrote:
> *@cloud-fan* commented on this pull request.
> ------------------------------
>
> In core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java
> <https://github.com/apache/spark/pull/23272#discussion_r240463496>:
>
> > @@ -283,6 +290,9 @@ private void advanceToNextPage() {
> }
> }
> }
> + if (pageToFree != null) {
> + freePage(pageToFree);
>
> the MapIterator.spill will be called by BytesToBytesMap.spill which will
> be called by other consumers.
>
> —
> You are receiving this because you were mentioned.
> Reply to this email directly, view it on GitHub
> <https://github.com/apache/spark/pull/23272#discussion_r240463496>, or mute
> the thread
>
<https://github.com/notifications/unsubscribe-auth/AAEM91AsLKsA5zS0gXEhip3GuUcVjJ7-ks5u3yfAgaJpZM4ZK2_Y>
> .
>
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
> 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 cores and 182000G mem.
> Enviroment:
> |spark.dynamicAllocation.minExecutors|5|
> |spark.dynamicAllocation.initialExecutors|30|
> |spark.dynamicAllocation.maxExecutors|400|
> |spark.executor.cores|4|
> |spark.executor.memory|20g|
>
>
> Stage description:
> 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 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]