You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "KaiXu (JIRA)" <ji...@apache.org> on 2016/11/05 13:09:58 UTC

[jira] [Created] (SPARK-18289) spark.util.collection.ExternalSorter leak memory when task force spilling in-memory map to disk

KaiXu created SPARK-18289:
-----------------------------

             Summary: spark.util.collection.ExternalSorter leak memory when task force spilling in-memory map to disk 
                 Key: SPARK-18289
                 URL: https://issues.apache.org/jira/browse/SPARK-18289
             Project: Spark
          Issue Type: Bug
          Components: Scheduler, Shuffle, Spark Core
    Affects Versions: 2.0.0
         Environment: CentOS7.2 kernel: 3.10.0-327.el7.x86_64
Hadoop2.7.1
Spark2.0.0 release version
Hive2.1 with patch HIVE-14029 (https://issues.apache.org/jira/browse/HIVE-14029)
Intel(R) Xeon(R) CPU E5-2699 v4 @ 2.20GHz
384GB memory
            Reporter: KaiXu


We use BigBench to test the performance of Hive on Spark2.0 on Intel(R) Xeon(R) CPU E5-2699 v4(1 master + 4 worker) with 3TB scale factor, we found 4 tasks failed when running q30. When checking in executor's log, we found it has memory leak when task force spilling in-memory map to disk.
notable configuration parameters:
spark.master=yarn
spark.deploy.mode=cluster
spark.driver.memory=4g;
spark.executor.cores=7;
spark.executor.memory=22g;
spark.yarn.executor.memoryOverhead=8192;
spark.executor.instances=50;
spark.storage.memoryFraction=0.01;
spark.memory.useLegacyMode=true;
spark.executor.extraJavaOptions=-XX:+UseParallelOldGC -XX:NewRatio=1 -XX:SurvivorRatio=1 -XX:ParallelGCThreads=4;

executor's log:
16/11/05 18:12:32 INFO exec.CommonJoinOperator: table 0 has 2000 rows for join key [973089]
16/11/05 18:12:32 INFO exec.CommonJoinOperator: table 0 has 4000 rows for join key [973089]
16/11/05 18:12:32 INFO exec.CommonJoinOperator: table 0 has 8000 rows for join key [973089]
16/11/05 18:12:32 INFO exec.CommonJoinOperator: table 0 has 16000 rows for join key [973089]
16/11/05 18:12:32 INFO exec.CommonJoinOperator: table 0 has 1000 rows for join key [972962]
16/11/05 18:12:32 INFO exec.CommonJoinOperator: table 0 has 2000 rows for join key [972962]
16/11/05 18:12:32 INFO exec.CommonJoinOperator: table 0 has 4000 rows for join key [972962]
16/11/05 18:12:32 INFO exec.CommonJoinOperator: table 0 has 8000 rows for join key [972962]
16/11/05 18:12:32 INFO exec.CommonJoinOperator: table 0 has 16000 rows for join key [972962]
16/11/05 18:12:32 INFO spark.SparkRecordHandler: processed 1051034 rows: used memory = 10801674800
16/11/05 18:12:32 INFO exec.JoinOperator: SKEWJOINFOLLOWUPJOBS:0, 
16/11/05 18:12:32 INFO exec.ReduceSinkOperator: RS[9]: records written - 1050990
16/11/05 18:12:32 INFO exec.ReduceSinkOperator: RECORDS_OUT_INTERMEDIATE:1050990, 
16/11/05 18:12:32 INFO collection.ExternalSorter: Task 6856 force spilling in-memory map to disk and  it will release 347.0 MB memory
16/11/05 18:12:32 WARN memory.TaskMemoryManager: leak 347.0 MB memory from org.apache.spark.util.collection.ExternalSorter@1bf0a514
16/11/05 18:12:32 ERROR executor.Executor: Exception in task 5978.0 in stage 1.0 (TID 6856)
java.lang.NullPointerException
	at org.apache.spark.util.collection.ExternalSorter$SpillReader.cleanup(ExternalSorter.scala:625)
	at org.apache.spark.util.collection.ExternalSorter$SpillReader.nextBatchStream(ExternalSorter.scala:540)
	at org.apache.spark.util.collection.ExternalSorter$SpillReader.<init>(ExternalSorter.scala:508)
	at org.apache.spark.util.collection.ExternalSorter$SpillableIterator.spill(ExternalSorter.scala:814)
	at org.apache.spark.util.collection.ExternalSorter.forceSpill(ExternalSorter.scala:254)
	at org.apache.spark.util.collection.Spillable.spill(Spillable.scala:111)
	at org.apache.spark.memory.TaskMemoryManager.acquireExecutionMemory(TaskMemoryManager.java:150)
	at org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:245)
	at org.apache.spark.memory.MemoryConsumer.allocateArray(MemoryConsumer.java:92)
	at org.apache.spark.shuffle.sort.ShuffleExternalSorter.growPointerArrayIfNecessary(ShuffleExternalSorter.java:333)
	at org.apache.spark.shuffle.sort.ShuffleExternalSorter.insertRecord(ShuffleExternalSorter.java:384)
	at org.apache.spark.shuffle.sort.UnsafeShuffleWriter.insertRecordIntoSorter(UnsafeShuffleWriter.java:235)
	at org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:162)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:79)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:47)
	at org.apache.spark.scheduler.Task.run(Task.scala:85)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
	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:745)
16/11/05 18:12:32 INFO executor.CoarseGrainedExecutorBackend: Got assigned task 7173
16/11/05 18:12:32 INFO executor.Executor: Running task 6294.0 in stage 1.0 (TID 7173)
16/11/05 18:12:32 INFO storage.ShuffleBlockFetcherIterator: Getting 878 non-empty blocks out of 878 blocks
16/11/05 18:12:32 INFO storage.ShuffleBlockFetcherIterator: Started 46 remote fetches in 17 ms
16/11/05 18:12:39 INFO executor.Executor: Finished task 6095.0 in stage 1.0 (TID 6973). 3681 bytes result sent to driver
16/11/05 18:12:39 INFO executor.CoarseGrainedExecutorBackend: Got assigned task 7418
16/11/05 18:12:39 INFO executor.Executor: Running task 6538.0 in stage 1.0 (TID 7418)
16/11/05 18:12:39 INFO storage.ShuffleBlockFetcherIterator: Getting 878 non-empty blocks out of 878 blocks
16/11/05 18:12:39 INFO storage.ShuffleBlockFetcherIterator: Started 46 remote fetches in 10 ms
16/11/05 18:12:39 INFO executor.Executor: Finished task 6010.0 in stage 1.0 (TID 6888). 3681 bytes result sent to driver
16/11/05 18:12:39 INFO executor.CoarseGrainedExecutorBackend: Got assigned task 7421
16/11/05 18:12:39 INFO executor.Executor: Running task 6541.0 in stage 1.0 (TID 7421)
16/11/05 18:12:39 INFO storage.ShuffleBlockFetcherIterator: Getting 878 non-empty blocks out of 878 blocks
16/11/05 18:12:39 INFO storage.ShuffleBlockFetcherIterator: Started 46 remote fetches in 10 ms
16/11/05 18:12:40 INFO executor.Executor: Finished task 6160.0 in stage 1.0 (TID 7038). 3681 bytes result sent to driver
16/11/05 18:12:40 INFO executor.CoarseGrainedExecutorBackend: Got assigned task 7424
16/11/05 18:12:40 INFO executor.Executor: Running task 6544.0 in stage 1.0 (TID 7424)
16/11/05 18:12:40 INFO storage.ShuffleBlockFetcherIterator: Getting 878 non-empty blocks out of 878 blocks
16/11/05 18:12:40 INFO storage.ShuffleBlockFetcherIterator: Started 46 remote fetches in 16 ms
16/11/05 18:12:40 INFO spark.SparkRecordHandler: maximum memory = 19685441536
16/11/05 18:12:40 INFO spark.SparkRecordHandler: conf classpath = [file:/mnt/disk6/yarn/nm/usercache/root/appcache/application_1478318297544_0031/container_1478318297544_0031_01_000022/__app__.jar, file:/mnt/disk6/yarn/nm/usercache/root/appcache/application_1478318297544_0031/container_1478318297544_0031_01_000022/__app__.jar, file:/mnt/disk6/yarn/nm/usercache/root/appcache/application_1478318297544_0031/container_1478318297544_0031_01_000022/./bigbenchqueriesmr.jar, file:/mnt/disk6/yarn/nm/usercache/root/appcache/application_1478318297544_0031/container_1478318297544_0031_01_000022/./hive-exec-2.2.0-SNAPSHOT.jar]
16/11/05 18:12:40 INFO spark.SparkRecordHandler: thread classpath = [file:/mnt/disk6/yarn/nm/usercache/root/appcache/application_1478318297544_0031/container_1478318297544_0031_01_000022/__app__.jar, file:/mnt/disk6/yarn/nm/usercache/root/appcache/application_1478318297544_0031/container_1478318297544_0031_01_000022/__app__.jar, file:/mnt/disk6/yarn/nm/usercache/root/appcache/application_1478318297544_0031/container_1478318297544_0031_01_000022/./bigbenchqueriesmr.jar, file:/mnt/disk6/yarn/nm/usercache/root/appcache/application_1478318297544_0031/container_1478318297544_0031_01_000022/./hive-exec-2.2.0-SNAPSHOT.jar]
16/11/05 18:12:40 INFO exec.Utilities: PLAN PATH = hdfs://eurus-master:8020/tmp/hive/root/6d511c4f-2750-49c3-894b-f5e78d1095c4/hive_2016-11-05_18-01-42_902_5968983454776048389-1/-mr-10002/b2c01f90-f0d1-43bb-bb80-f0e3a874c390/reduce.xml
16/11/05 18:12:40 INFO exec.SerializationUtilities: Deserializing ReduceWork using kryo
16/11/05 18:12:40 INFO exec.Utilities: Deserialized plan (via FILE) - name: Reducer 2 size: 3.91KB


16/11/05 18:21:02 INFO exec.CommonJoinOperator: table 0 has 8000 rows for join key [802585]
16/11/05 18:21:02 INFO exec.CommonJoinOperator: table 0 has 16000 rows for join key [802585]
16/11/05 18:21:02 INFO exec.CommonJoinOperator: table 0 has 1000 rows for join key [825039]
16/11/05 18:21:02 INFO exec.CommonJoinOperator: table 0 has 2000 rows for join key [825039]
16/11/05 18:21:02 INFO exec.CommonJoinOperator: table 0 has 4000 rows for join key [825039]
16/11/05 18:21:02 INFO exec.CommonJoinOperator: table 0 has 8000 rows for join key [825039]
16/11/05 18:21:02 INFO exec.CommonJoinOperator: table 0 has 16000 rows for join key [825039]
16/11/05 18:21:02 INFO collection.ExternalSorter: Task 16349 force spilling in-memory map to disk and  it will release 360.5 MB memory
16/11/05 18:21:02 WARN memory.TaskMemoryManager: leak 360.5 MB memory from org.apache.spark.util.collection.ExternalSorter@5879fd17
16/11/05 18:21:02 ERROR executor.Executor: Exception in task 15468.0 in stage 1.0 (TID 16349)
java.lang.NullPointerException
	at org.apache.spark.util.collection.ExternalSorter$SpillReader.cleanup(ExternalSorter.scala:625)
	at org.apache.spark.util.collection.ExternalSorter$SpillReader.nextBatchStream(ExternalSorter.scala:540)
	at org.apache.spark.util.collection.ExternalSorter$SpillReader.<init>(ExternalSorter.scala:508)
	at org.apache.spark.util.collection.ExternalSorter$SpillableIterator.spill(ExternalSorter.scala:814)
	at org.apache.spark.util.collection.ExternalSorter.forceSpill(ExternalSorter.scala:254)
	at org.apache.spark.util.collection.Spillable.spill(Spillable.scala:111)
	at org.apache.spark.memory.TaskMemoryManager.acquireExecutionMemory(TaskMemoryManager.java:150)
	at org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:245)
	at org.apache.spark.memory.MemoryConsumer.allocateArray(MemoryConsumer.java:92)
	at org.apache.spark.shuffle.sort.ShuffleExternalSorter.growPointerArrayIfNecessary(ShuffleExternalSorter.java:333)
	at org.apache.spark.shuffle.sort.ShuffleExternalSorter.insertRecord(ShuffleExternalSorter.java:384)
	at org.apache.spark.shuffle.sort.UnsafeShuffleWriter.insertRecordIntoSorter(UnsafeShuffleWriter.java:235)
	at org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:162)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:79)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:47)
	at org.apache.spark.scheduler.Task.run(Task.scala:85)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
	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:745)
16/11/05 18:21:02 INFO executor.CoarseGrainedExecutorBackend: Got assigned task 16611
16/11/05 18:21:02 INFO executor.Executor: Running task 15730.0 in stage 1.0 (TID 16611)
16/11/05 18:21:02 INFO exec.CommonJoinOperator: table 0 has 1000 rows for join key [667774]
16/11/05 18:21:02 INFO exec.CommonJoinOperator: table 0 has 2000 rows for join key [667774]
16/11/05 18:21:02 INFO exec.CommonJoinOperator: table 0 has 4000 rows for join key [667774]
16/11/05 18:21:02 INFO exec.CommonJoinOperator: table 0 has 8000 rows for join key [667774]
16/11/05 18:21:02 INFO storage.ShuffleBlockFetcherIterator: Getting 877 non-empty blocks out of 878 blocks
16/11/05 18:21:02 INFO exec.CommonJoinOperator: table 0 has 16000 rows for join key [667774]
16/11/05 18:21:02 INFO storage.ShuffleBlockFetcherIterator: Started 46 remote fetches in 9 ms
16/11/05 18:21:02 INFO exec.CommonJoinOperator: table 0 has 1000 rows for join key [825073]
16/11/05 18:21:02 INFO exec.CommonJoinOperator: table 0 has 2000 rows for join key [825073]
16/11/05 18:21:02 INFO exec.CommonJoinOperator: table 0 has 4000 rows for join key [825073]
16/11/05 18:21:02 INFO exec.CommonJoinOperator: table 0 has 8000 rows for join key [825073]
16/11/05 18:21:02 INFO exec.CommonJoinOperator: table 0 has 16000 rows for join key [825073]
16/11/05 18:21:02 INFO exec.CommonJoinOperator: table 0 has 1000 rows for join key [847527]
16/11/05 18:21:02 INFO exec.CommonJoinOperator: table 0 has 2000 rows for join key [847527]



16/11/05 18:12:22 INFO exec.CommonJoinOperator: table 0 has 1000 rows for join key [972725]
16/11/05 18:12:22 INFO exec.CommonJoinOperator: table 0 has 2000 rows for join key [972725]
16/11/05 18:12:22 INFO exec.CommonJoinOperator: table 0 has 4000 rows for join key [972725]
16/11/05 18:12:22 INFO exec.CommonJoinOperator: table 0 has 8000 rows for join key [972725]
16/11/05 18:12:22 INFO exec.CommonJoinOperator: table 0 has 16000 rows for join key [972725]
16/11/05 18:12:22 INFO collection.ExternalSorter: Task 6619 force spilling in-memory map to disk and  it will release 357.9 MB memory
16/11/05 18:12:22 WARN memory.TaskMemoryManager: leak 357.9 MB memory from org.apache.spark.util.collection.ExternalSorter@71b91fb4
16/11/05 18:12:22 ERROR executor.Executor: Exception in task 5741.0 in stage 1.0 (TID 6619)
java.lang.NullPointerException
	at org.apache.spark.util.collection.ExternalSorter$SpillReader.cleanup(ExternalSorter.scala:625)
	at org.apache.spark.util.collection.ExternalSorter$SpillReader.nextBatchStream(ExternalSorter.scala:540)
	at org.apache.spark.util.collection.ExternalSorter$SpillReader.<init>(ExternalSorter.scala:508)
	at org.apache.spark.util.collection.ExternalSorter$SpillableIterator.spill(ExternalSorter.scala:814)
	at org.apache.spark.util.collection.ExternalSorter.forceSpill(ExternalSorter.scala:254)
	at org.apache.spark.util.collection.Spillable.spill(Spillable.scala:111)
	at org.apache.spark.memory.TaskMemoryManager.acquireExecutionMemory(TaskMemoryManager.java:150)
	at org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:245)
	at org.apache.spark.memory.MemoryConsumer.allocateArray(MemoryConsumer.java:92)
	at org.apache.spark.shuffle.sort.ShuffleExternalSorter.growPointerArrayIfNecessary(ShuffleExternalSorter.java:333)
	at org.apache.spark.shuffle.sort.ShuffleExternalSorter.insertRecord(ShuffleExternalSorter.java:384)
	at org.apache.spark.shuffle.sort.UnsafeShuffleWriter.insertRecordIntoSorter(UnsafeShuffleWriter.java:235)
	at org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:162)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:79)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:47)
	at org.apache.spark.scheduler.Task.run(Task.scala:85)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
	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:745)
16/11/05 18:12:22 INFO executor.CoarseGrainedExecutorBackend: Got assigned task 7053
16/11/05 18:12:22 INFO executor.Executor: Running task 6175.0 in stage 1.0 (TID 7053)
16/11/05 18:12:22 INFO storage.ShuffleBlockFetcherIterator: Getting 878 non-empty blocks out of 878 blocks
16/11/05 18:12:22 INFO storage.ShuffleBlockFetcherIterator: Started 46 remote fetches in 7 ms
16/11/05 18:12:25 INFO spark.SparkRecordHandler: maximum memory = 19685441536
16/11/05 18:12:25 INFO spark.SparkRecordHandler: conf classpath = [file:/mnt/disk1/yarn/nm/usercache/root/appcache/application_1478318297544_0031/container_1478318297544_0031_01_000030/__app__.jar, file:/mnt/disk1/yarn/nm/usercache/root/appcache/application_1478318297544_0031/container_1478318297544_0031_01_000030/__app__.jar, file:/mnt/disk1/yarn/nm/usercache/root/appcache/application_1478318297544_0031/container_1478318297544_0031_01_000030/./bigbenchqueriesmr.jar, file:/mnt/disk1/yarn/nm/usercache/root/appcache/application_1478318297544_0031/container_1478318297544_0031_01_000030/./hive-exec-2.2.0-SNAPSHOT.jar]
16/11/05 18:12:25 INFO spark.SparkRecordHandler: thread classpath = [file:/mnt/disk1/yarn/nm/usercache/root/appcache/application_1478318297544_0031/container_1478318297544_0031_01_000030/__app__.jar, file:/mnt/disk1/yarn/nm/usercache/root/appcache/application_1478318297544_0031/container_1478318297544_0031_01_000030/__app__.jar, file:/mnt/disk1/yarn/nm/usercache/root/appcache/application_1478318297544_0031/container_1478318297544_0031_01_000030/./bigbenchqueriesmr.jar, file:/mnt/disk1/yarn/nm/usercache/root/appcache/application_1478318297544_0031/container_1478318297544_0031_01_000030/./hive-exec-2.2.0-SNAPSHOT.jar]
16/11/05 18:12:25 INFO exec.Utilities: PLAN PATH = hdfs://eurus-master:8020/tmp/hive/root/6d511c4f-2750-49c3-894b-f5e78d1095c4/hive_2016-11-05_18-01-42_902_5968983454776048389-1/-mr-10002/b2c01f90-f0d1-43bb-bb80-f0e3a874c390/reduce.xml
16/11/05 18:12:25 INFO exec.SerializationUtilities: Deserializing ReduceWork using kryo
16/11/05 18:12:25 INFO exec.Utilities: Deserialized plan (via FILE) - name: Reducer 2 size: 3.91KB



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org