[ https://issues.apache.org/jira/browse/SPARK-18289?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
KaiXu updated SPARK-18289: -------------------------- Description: 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; hive.exec.reducers.bytes.per.reducer=128000000; hive.exec.reducers.max=1000000000; hive.mapjoin.smalltable.filesize=1000000000; hive.auto.convert.join.noconditionaltask.size=1000000000; 2016-11-05 18:12:22,754 Stage-0_0: 878/878 Finished Stage-1_0: 5847(+329,-1)/22488 Stage-2_0: 0/12354 Stage-3_0: 0/12354 Stage-4_0: 0/6177 Stage-5_0: 0/1 2016-11-05 18:12:23,760 Stage-0_0: 878/878 Finished Stage-1_0: 5872(+329,-1)/22488 Stage-2_0: 0/12354 Stage-3_0: 0/12354 Stage-4_0: 0/6177 Stage-5_0: 0/1 2016-11-05 18:12:24,766 Stage-0_0: 878/878 Finished Stage-1_0: 5886(+329,-1)/22488 Stage-2_0: 0/12354 Stage-3_0: 0/12354 Stage-4_0: 0/6177 Stage-5_0: 0/1 2016-11-05 18:12:25,772 Stage-0_0: 878/878 Finished Stage-1_0: 5892(+329,-1)/22488 Stage-2_0: 0/12354 Stage-3_0: 0/12354 Stage-4_0: 0/6177 Stage-5_0: 0/1 2016-11-05 18:12:32,810 Stage-0_0: 878/878 Finished Stage-1_0: 5972(+329,-2)/22488 Stage-2_0: 0/12354 Stage-3_0: 0/12354 Stage-4_0: 0/6177 Stage-5_0: 0/1 2016-11-05 18:12:33,815 Stage-0_0: 878/878 Finished Stage-1_0: 6007(+329,-2)/22488 Stage-2_0: 0/12354 Stage-3_0: 0/12354 Stage-4_0: 0/6177 Stage-5_0: 0/1 2016-11-05 18:12:34,821 Stage-0_0: 878/878 Finished Stage-1_0: 6048(+329,-2)/22488 Stage-2_0: 0/12354 Stage-3_0: 0/12354 Stage-4_0: 0/6177 Stage-5_0: 0/1 2016-11-05 18:12:35,827 Stage-0_0: 878/878 Finished Stage-1_0: 6089(+329,-2)/22488 Stage-2_0: 0/12354 Stage-3_0: 0/12354 Stage-4_0: 0/6177 Stage-5_0: 0/1 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 was: 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; hive.exec.reducers.bytes.per.reducer=128000000; hive.exec.reducers.max=1000000000; hive.mapjoin.smalltable.filesize=1000000000; hive.auto.convert.join.noconditionaltask.size=1000000000; 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 > 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 > Labels: hive, spark2.0.0 > > 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; > hive.exec.reducers.bytes.per.reducer=128000000; > hive.exec.reducers.max=1000000000; > hive.mapjoin.smalltable.filesize=1000000000; > hive.auto.convert.join.noconditionaltask.size=1000000000; > 2016-11-05 18:12:22,754 Stage-0_0: 878/878 Finished Stage-1_0: > 5847(+329,-1)/22488 Stage-2_0: 0/12354 Stage-3_0: 0/12354 > Stage-4_0: 0/6177 Stage-5_0: 0/1 > 2016-11-05 18:12:23,760 Stage-0_0: 878/878 Finished Stage-1_0: > 5872(+329,-1)/22488 Stage-2_0: 0/12354 Stage-3_0: 0/12354 > Stage-4_0: 0/6177 Stage-5_0: 0/1 > 2016-11-05 18:12:24,766 Stage-0_0: 878/878 Finished Stage-1_0: > 5886(+329,-1)/22488 Stage-2_0: 0/12354 Stage-3_0: 0/12354 > Stage-4_0: 0/6177 Stage-5_0: 0/1 > 2016-11-05 18:12:25,772 Stage-0_0: 878/878 Finished Stage-1_0: > 5892(+329,-1)/22488 Stage-2_0: 0/12354 Stage-3_0: 0/12354 > Stage-4_0: 0/6177 Stage-5_0: 0/1 > 2016-11-05 18:12:32,810 Stage-0_0: 878/878 Finished Stage-1_0: > 5972(+329,-2)/22488 Stage-2_0: 0/12354 Stage-3_0: 0/12354 > Stage-4_0: 0/6177 Stage-5_0: 0/1 > 2016-11-05 18:12:33,815 Stage-0_0: 878/878 Finished Stage-1_0: > 6007(+329,-2)/22488 Stage-2_0: 0/12354 Stage-3_0: 0/12354 > Stage-4_0: 0/6177 Stage-5_0: 0/1 > 2016-11-05 18:12:34,821 Stage-0_0: 878/878 Finished Stage-1_0: > 6048(+329,-2)/22488 Stage-2_0: 0/12354 Stage-3_0: 0/12354 > Stage-4_0: 0/6177 Stage-5_0: 0/1 > 2016-11-05 18:12:35,827 Stage-0_0: 878/878 Finished Stage-1_0: > 6089(+329,-2)/22488 Stage-2_0: 0/12354 Stage-3_0: 0/12354 > Stage-4_0: 0/6177 Stage-5_0: 0/1 > 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-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org