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

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

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

Sean Owen resolved SPARK-18289.
-------------------------------
    Resolution: Duplicate

> 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
> 2016-11-05 18:17:16,348 Stage-0_0: 878/878 Finished     Stage-1_0: 11666(+329,-3)/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:17:17,354 Stage-0_0: 878/878 Finished     Stage-1_0: 11691(+329,-3)/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:17:18,359 Stage-0_0: 878/878 Finished     Stage-1_0: 11714(+329,-3)/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:17:19,364 Stage-0_0: 878/878 Finished     Stage-1_0: 11738(+329,-3)/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:21:03,468 Stage-0_0: 878/878 Finished     Stage-1_0: 15413(+329,-4)/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:21:04,473 Stage-0_0: 878/878 Finished     Stage-1_0: 15447(+329,-4)/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:21:05,477 Stage-0_0: 878/878 Finished     Stage-1_0: 15479(+329,-4)/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:21:06,482 Stage-0_0: 878/878 Finished     Stage-1_0: 15511(+329,-4)/22488 Stage-2_0: 0/12354      Stage-3_0: 0/12354      Stage-4_0: 0/6177       Stage-5_0: 0/1
> failed task log:
> 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)
> executor's log:
> 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
> 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]



--
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