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

Andrew Or resolved SPARK-12155.
-------------------------------
       Resolution: Fixed
    Fix Version/s: 1.6.0

> Execution OOM after a relative large dataset cached in the cluster.
> -------------------------------------------------------------------
>
>                 Key: SPARK-12155
>                 URL: https://issues.apache.org/jira/browse/SPARK-12155
>             Project: Spark
>          Issue Type: Bug
>          Components: Spark Core, SQL
>            Reporter: Yin Huai
>            Assignee: Josh Rosen
>            Priority: Blocker
>             Fix For: 1.6.0
>
>
> I have a cluster with relative 80GB of mem. Then, I cached a 43GB dataframe. 
> When I start to consume the query. I got the following exception (I added 
> more logs to the code).
> {code}
> 15/12/05 00:33:43 INFO UnifiedMemoryManager: Creating UnifedMemoryManager for 
> 4 cores with 16929521664 maxMemory, 8464760832 storageRegionSize.
> 15/12/05 01:20:50 INFO MemoryStore: Ensuring 1048576 bytes of free space for 
> block rdd_94_37(free: 3253659951, max: 16798973952)
> 15/12/05 01:20:50 INFO MemoryStore: Ensuring 5142008 bytes of free space for 
> block rdd_94_37(free: 3252611375, max: 16798973952)
> 15/12/05 01:20:50 INFO Executor: Finished task 36.0 in stage 4.0 (TID 109). 
> 3028 bytes result sent to driver
> 15/12/05 01:20:50 INFO MemoryStore: Ensuring 98948238 bytes of free space for 
> block rdd_94_37(free: 3314840375, max: 16866344960)
> 15/12/05 01:20:50 INFO MemoryStore: Ensuring 98675713 bytes of free space for 
> block rdd_94_37(free: 3215892137, max: 16866344960)
> 15/12/05 01:20:50 INFO MemoryStore: Ensuring 197347565 bytes of free space 
> for block rdd_94_37(free: 3117216424, max: 16866344960)
> 15/12/05 01:20:50 INFO MemoryStore: Ensuring 295995553 bytes of free space 
> for block rdd_94_37(free: 2919868859, max: 16866344960)
> 15/12/05 01:20:51 INFO MemoryStore: Ensuring 394728479 bytes of free space 
> for block rdd_94_37(free: 2687050010, max: 16929521664)
> 15/12/05 01:20:51 INFO Executor: Finished task 32.0 in stage 4.0 (TID 106). 
> 3028 bytes result sent to driver
> 15/12/05 01:20:51 INFO MemoryStore: Ensuring 591258816 bytes of free space 
> for block rdd_94_37(free: 2292321531, max: 16929521664)
> 15/12/05 01:20:51 INFO MemoryStore: Ensuring 901645182 bytes of free space 
> for block rdd_94_37(free: 1701062715, max: 16929521664)
> 15/12/05 01:20:52 INFO MemoryStore: Ensuring 1302179076 bytes of free space 
> for block rdd_94_37(free: 799417533, max: 16929521664)
> 15/12/05 01:20:52 INFO MemoryStore: Will not store rdd_94_37 as it would 
> require dropping another block from the same RDD
> 15/12/05 01:20:52 WARN MemoryStore: Not enough space to cache rdd_94_37 in 
> memory! (computed 2.4 GB so far)
> 15/12/05 01:20:52 INFO MemoryStore: Memory use = 12.6 GB (blocks) + 2.4 GB 
> (scratch space shared across 13 tasks(s)) = 15.0 GB. Storage limit = 15.8 GB.
> 15/12/05 01:20:52 INFO BlockManager: Found block rdd_94_37 locally
> 15/12/05 01:20:52 INFO UnifiedMemoryManager: Try to acquire 262144 bytes 
> memory. But, on-heap execution memory poll only has 0 bytes free memory.
> 15/12/05 01:20:52 INFO UnifiedMemoryManager: memoryReclaimableFromStorage 
> 8464760832, storageMemoryPool.poolSize 16929521664, storageRegionSize 
> 8464760832.
> 15/12/05 01:20:52 INFO UnifiedMemoryManager: Try to reclaim memory space from 
> storage memory pool.
> 15/12/05 01:20:52 INFO StorageMemoryPool: Claiming 262144 bytes free memory 
> space from StorageMemoryPool.
> 15/12/05 01:20:52 INFO UnifiedMemoryManager: Reclaimed 262144 bytes of memory 
> from storage memory pool.Adding them back to onHeapExecutionMemoryPool.
> 15/12/05 01:20:52 INFO UnifiedMemoryManager: Try to acquire 67108864 bytes 
> memory. But, on-heap execution memory poll only has 0 bytes free memory.
> 15/12/05 01:20:52 INFO UnifiedMemoryManager: memoryReclaimableFromStorage 
> 8464498688, storageMemoryPool.poolSize 16929259520, storageRegionSize 
> 8464760832.
> 15/12/05 01:20:52 INFO UnifiedMemoryManager: Try to reclaim memory space from 
> storage memory pool.
> 15/12/05 01:20:52 INFO StorageMemoryPool: Claiming 67108864 bytes free memory 
> space from StorageMemoryPool.
> 15/12/05 01:20:52 INFO UnifiedMemoryManager: Reclaimed 67108864 bytes of 
> memory from storage memory pool.Adding them back to onHeapExecutionMemoryPool.
> 15/12/05 01:20:54 INFO Executor: Finished task 37.0 in stage 4.0 (TID 110). 
> 3077 bytes result sent to driver
> 15/12/05 01:20:56 INFO CoarseGrainedExecutorBackend: Got assigned task 120
> 15/12/05 01:20:56 INFO Executor: Running task 1.0 in stage 5.0 (TID 120)
> 15/12/05 01:20:56 INFO CoarseGrainedExecutorBackend: Got assigned task 124
> 15/12/05 01:20:56 INFO CoarseGrainedExecutorBackend: Got assigned task 128
> 15/12/05 01:20:56 INFO CoarseGrainedExecutorBackend: Got assigned task 132
> 15/12/05 01:20:56 INFO Executor: Running task 9.0 in stage 5.0 (TID 128)
> 15/12/05 01:20:56 INFO Executor: Running task 13.0 in stage 5.0 (TID 132)
> 15/12/05 01:20:56 INFO Executor: Running task 5.0 in stage 5.0 (TID 124)
> 15/12/05 01:20:56 INFO MapOutputTrackerWorker: Updating epoch to 2 and 
> clearing cache
> 15/12/05 01:20:56 INFO TorrentBroadcast: Started reading broadcast variable 6
> 15/12/05 01:20:56 INFO MemoryStore: Ensuring 9471 bytes of free space for 
> block broadcast_6_piece0(free: 3384207663, max: 16929521664)
> 15/12/05 01:20:56 INFO MemoryStore: Block broadcast_6_piece0 stored as bytes 
> in memory (estimated size 9.2 KB, free 12.6 GB)
> 15/12/05 01:20:56 INFO TorrentBroadcast: Reading broadcast variable 6 took 5 
> ms
> 15/12/05 01:20:56 INFO MemoryStore: Ensuring 1048576 bytes of free space for 
> block broadcast_6(free: 3384198192, max: 16929521664)
> 15/12/05 01:20:56 INFO MemoryStore: Ensuring 22032 bytes of free space for 
> block broadcast_6(free: 3384198192, max: 16929521664)
> 15/12/05 01:20:56 INFO MemoryStore: Block broadcast_6 stored as values in 
> memory (estimated size 21.5 KB, free 12.6 GB)
> 15/12/05 01:20:56 INFO MapOutputTrackerWorker: Don't have map outputs for 
> shuffle 1, fetching them
> 15/12/05 01:20:56 INFO MapOutputTrackerWorker: Don't have map outputs for 
> shuffle 1, fetching them
> 15/12/05 01:20:56 INFO MapOutputTrackerWorker: Don't have map outputs for 
> shuffle 1, fetching them
> 15/12/05 01:20:56 INFO MapOutputTrackerWorker: Don't have map outputs for 
> shuffle 1, fetching them
> 15/12/05 01:20:56 INFO MapOutputTrackerWorker: Doing the fetch; tracker 
> endpoint = NettyRpcEndpointRef(spark://MapOutputTracker@10.0.202.130:56969)
> 15/12/05 01:20:56 INFO MapOutputTrackerWorker: Got the output locations
> 15/12/05 01:20:56 INFO ShuffleBlockFetcherIterator: Getting 43 non-empty 
> blocks out of 43 blocks
> 15/12/05 01:20:56 INFO ShuffleBlockFetcherIterator: Getting 43 non-empty 
> blocks out of 43 blocks
> 15/12/05 01:20:56 INFO ShuffleBlockFetcherIterator: Getting 43 non-empty 
> blocks out of 43 blocks
> 15/12/05 01:20:56 INFO ShuffleBlockFetcherIterator: Getting 43 non-empty 
> blocks out of 43 blocks
> 15/12/05 01:20:56 INFO ShuffleBlockFetcherIterator: Started 3 remote fetches 
> in 41 ms
> 15/12/05 01:20:56 INFO ShuffleBlockFetcherIterator: Started 3 remote fetches 
> in 41 ms
> 15/12/05 01:20:56 INFO ShuffleBlockFetcherIterator: Started 3 remote fetches 
> in 40 ms
> 15/12/05 01:20:56 INFO ShuffleBlockFetcherIterator: Started 3 remote fetches 
> in 41 ms
> 15/12/05 01:20:56 INFO UnifiedMemoryManager: Try to acquire 67108864 bytes 
> memory. But, on-heap execution memory poll only has 66846720 bytes free 
> memory.
> 15/12/05 01:20:56 INFO UnifiedMemoryManager: memoryReclaimableFromStorage 
> 8397389824, storageMemoryPool.poolSize 16862150656, storageRegionSize 
> 8464760832.
> 15/12/05 01:20:56 INFO UnifiedMemoryManager: Try to reclaim memory space from 
> storage memory pool.
> 15/12/05 01:20:56 INFO StorageMemoryPool: Claiming 262144 bytes free memory 
> space from StorageMemoryPool.
> 15/12/05 01:20:56 INFO UnifiedMemoryManager: Reclaimed 262144 bytes of memory 
> from storage memory pool.Adding them back to onHeapExecutionMemoryPool.
> 15/12/05 01:20:56 INFO UnifiedMemoryManager: Try to acquire 67108864 bytes 
> memory. But, on-heap execution memory poll only has 33554432 bytes free 
> memory.
> 15/12/05 01:20:56 INFO UnifiedMemoryManager: memoryReclaimableFromStorage 
> 8397127680, storageMemoryPool.poolSize 16861888512, storageRegionSize 
> 8464760832.
> 15/12/05 01:20:56 INFO UnifiedMemoryManager: Try to reclaim memory space from 
> storage memory pool.
> 15/12/05 01:20:56 INFO StorageMemoryPool: Claiming 33554432 bytes free memory 
> space from StorageMemoryPool.
> 15/12/05 01:20:56 INFO UnifiedMemoryManager: Reclaimed 33554432 bytes of 
> memory from storage memory pool.Adding them back to onHeapExecutionMemoryPool.
> 15/12/05 01:20:56 INFO GenerateMutableProjection: Code generated in 9.602791 
> ms
> 15/12/05 01:20:56 INFO GenerateMutableProjection: Code generated in 12.7135 ms
> 15/12/05 01:20:56 INFO Executor: Finished task 13.0 in stage 5.0 (TID 132). 
> 2271 bytes result sent to driver
> 15/12/05 01:20:56 INFO Executor: Finished task 9.0 in stage 5.0 (TID 128). 
> 2320 bytes result sent to driver
> 15/12/05 01:20:56 INFO CoarseGrainedExecutorBackend: Got assigned task 136
> 15/12/05 01:20:56 INFO CoarseGrainedExecutorBackend: Got assigned task 137
> 15/12/05 01:20:56 INFO Executor: Running task 17.0 in stage 5.0 (TID 136)
> 15/12/05 01:20:56 INFO ShuffleBlockFetcherIterator: Getting 43 non-empty 
> blocks out of 43 blocks
> 15/12/05 01:20:56 INFO ShuffleBlockFetcherIterator: Started 3 remote fetches 
> in 1 ms
> 15/12/05 01:20:56 INFO UnifiedMemoryManager: Try to acquire 67108864 bytes 
> memory. But, on-heap execution memory poll only has 16515072 bytes free 
> memory.
> 15/12/05 01:20:56 INFO UnifiedMemoryManager: memoryReclaimableFromStorage 
> 8363573248, storageMemoryPool.poolSize 16828334080, storageRegionSize 
> 8464760832.
> 15/12/05 01:20:56 INFO UnifiedMemoryManager: Try to reclaim memory space from 
> storage memory pool.
> 15/12/05 01:20:56 INFO StorageMemoryPool: Claiming 50593792 bytes free memory 
> space from StorageMemoryPool.
> 15/12/05 01:20:56 INFO UnifiedMemoryManager: Reclaimed 50593792 bytes of 
> memory from storage memory pool.Adding them back to onHeapExecutionMemoryPool.
> 15/12/05 01:20:56 INFO Executor: Running task 18.0 in stage 5.0 (TID 137)
> 15/12/05 01:20:56 INFO GenerateUnsafeProjection: Code generated in 30.25836 ms
> 15/12/05 01:20:56 INFO ShuffleBlockFetcherIterator: Getting 43 non-empty 
> blocks out of 43 blocks
> 15/12/05 01:20:56 INFO ShuffleBlockFetcherIterator: Started 3 remote fetches 
> in 2 ms
> 15/12/05 01:20:56 INFO UnifiedMemoryManager: Try to acquire 67108864 bytes 
> memory. But, on-heap execution memory poll only has 16515072 bytes free 
> memory.
> 15/12/05 01:20:56 INFO UnifiedMemoryManager: memoryReclaimableFromStorage 
> 8312979456, storageMemoryPool.poolSize 16777740288, storageRegionSize 
> 8464760832.
> 15/12/05 01:20:56 INFO UnifiedMemoryManager: Try to reclaim memory space from 
> storage memory pool.
> 15/12/05 01:20:56 INFO StorageMemoryPool: Claiming 50593792 bytes free memory 
> space from StorageMemoryPool.
> 15/12/05 01:20:56 INFO UnifiedMemoryManager: Reclaimed 50593792 bytes of 
> memory from storage memory pool.Adding them back to onHeapExecutionMemoryPool.
> 15/12/05 01:20:56 INFO GenerateUnsafeRowJoiner: Code generated in 19.615021 ms
> 15/12/05 01:20:57 INFO GenerateUnsafeProjection: Code generated in 23.149594 
> ms
> 15/12/05 01:20:57 INFO TaskMemoryManager: Memory used in task 136
> 15/12/05 01:20:57 INFO TaskMemoryManager: Acquired by 
> org.apache.spark.unsafe.map.BytesToBytesMap@5ac6b585: 48.3 MB
> 15/12/05 01:20:57 INFO TaskMemoryManager: 0 bytes of memory were used by task 
> 136 but are not associated with specific consumers
> 15/12/05 01:20:57 INFO TaskMemoryManager: 185597952 bytes of memory are used 
> for execution and 13545345504 bytes of memory are used for storage
> 15/12/05 01:20:57 INFO TaskMemoryManager: Memory used in task 124
> 15/12/05 01:20:57 INFO TaskMemoryManager: Acquired by 
> org.apache.spark.unsafe.map.BytesToBytesMap@30015a6a: 48.3 MB
> 15/12/05 01:20:57 INFO TaskMemoryManager: 0 bytes of memory were used by task 
> 124 but are not associated with specific consumers
> 15/12/05 01:20:57 INFO TaskMemoryManager: 185597952 bytes of memory are used 
> for execution and 13545345504 bytes of memory are used for storage
> 15/12/05 01:20:57 INFO UnifiedMemoryManager: Try to acquire 67108864 bytes 
> memory. But, on-heap execution memory poll only has 16515072 bytes free 
> memory.
> 15/12/05 01:20:57 INFO UnifiedMemoryManager: memoryReclaimableFromStorage 
> 8262385664, storageMemoryPool.poolSize 16727146496, storageRegionSize 
> 8464760832.
> 15/12/05 01:20:57 INFO UnifiedMemoryManager: Try to reclaim memory space from 
> storage memory pool.
> 15/12/05 01:20:57 INFO StorageMemoryPool: Claiming 50593792 bytes free memory 
> space from StorageMemoryPool.
> 15/12/05 01:20:57 INFO UnifiedMemoryManager: Reclaimed 50593792 bytes of 
> memory from storage memory pool.Adding them back to onHeapExecutionMemoryPool.
> 15/12/05 01:20:57 INFO TaskMemoryManager: Memory used in task 137
> 15/12/05 01:20:57 INFO TaskMemoryManager: Acquired by 
> org.apache.spark.unsafe.map.BytesToBytesMap@a9691e0: 48.3 MB
> 15/12/05 01:20:57 WARN TaskMemoryManager: leak 48.3 MB memory from 
> org.apache.spark.unsafe.map.BytesToBytesMap@5ac6b585
> 15/12/05 01:20:57 INFO TaskMemoryManager: 0 bytes of memory were used by task 
> 137 but are not associated with specific consumers
> 15/12/05 01:20:57 INFO TaskMemoryManager: 215023616 bytes of memory are used 
> for execution and 13545345504 bytes of memory are used for storage
> 15/12/05 01:20:57 WARN TaskMemoryManager: leak 48.3 MB memory from 
> org.apache.spark.unsafe.map.BytesToBytesMap@a9691e0
> 15/12/05 01:20:57 ERROR Executor: Managed memory leak detected; size = 
> 50593792 bytes, TID = 136
> 15/12/05 01:20:57 ERROR Executor: Managed memory leak detected; size = 
> 50593792 bytes, TID = 137
> 15/12/05 01:20:57 WARN TaskMemoryManager: leak 48.3 MB memory from 
> org.apache.spark.unsafe.map.BytesToBytesMap@30015a6a
> 15/12/05 01:20:57 ERROR Executor: Managed memory leak detected; size = 
> 50593792 bytes, TID = 124
> 15/12/05 01:20:57 ERROR Executor: Exception in task 18.0 in stage 5.0 (TID 
> 137)
> java.lang.OutOfMemoryError: Unable to acquire 262144 bytes of memory, got 0
>       at 
> org.apache.spark.memory.MemoryConsumer.allocateArray(MemoryConsumer.java:91)
>       at 
> org.apache.spark.unsafe.map.BytesToBytesMap.allocate(BytesToBytesMap.java:735)
>       at 
> org.apache.spark.unsafe.map.BytesToBytesMap.<init>(BytesToBytesMap.java:197)
>       at 
> org.apache.spark.unsafe.map.BytesToBytesMap.<init>(BytesToBytesMap.java:212)
>       at 
> org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap.<init>(UnsafeFixedWidthAggregationMap.java:103)
>       at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.<init>(TungstenAggregationIterator.scala:483)
>       at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:95)
>       at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:86)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73)
>       at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
>       at org.apache.spark.scheduler.Task.run(Task.scala:89)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:213)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> 15/12/05 01:20:57 ERROR Executor: Exception in task 17.0 in stage 5.0 (TID 
> 136)
> java.lang.OutOfMemoryError: Unable to acquire 262144 bytes of memory, got 0
>       at 
> org.apache.spark.memory.MemoryConsumer.allocateArray(MemoryConsumer.java:91)
>       at 
> org.apache.spark.unsafe.map.BytesToBytesMap.allocate(BytesToBytesMap.java:735)
>       at 
> org.apache.spark.unsafe.map.BytesToBytesMap.<init>(BytesToBytesMap.java:197)
>       at 
> org.apache.spark.unsafe.map.BytesToBytesMap.<init>(BytesToBytesMap.java:212)
>       at 
> org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap.<init>(UnsafeFixedWidthAggregationMap.java:103)
>       at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.<init>(TungstenAggregationIterator.scala:483)
>       at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:95)
>       at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:86)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73)
>       at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
>       at org.apache.spark.scheduler.Task.run(Task.scala:89)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:213)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> 15/12/05 01:20:57 ERROR Executor: Exception in task 5.0 in stage 5.0 (TID 124)
> java.lang.OutOfMemoryError: Unable to acquire 262144 bytes of memory, got 0
>       at 
> org.apache.spark.memory.MemoryConsumer.allocateArray(MemoryConsumer.java:91)
>       at 
> org.apache.spark.unsafe.map.BytesToBytesMap.allocate(BytesToBytesMap.java:735)
>       at 
> org.apache.spark.unsafe.map.BytesToBytesMap.<init>(BytesToBytesMap.java:197)
>       at 
> org.apache.spark.unsafe.map.BytesToBytesMap.<init>(BytesToBytesMap.java:212)
>       at 
> org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap.<init>(UnsafeFixedWidthAggregationMap.java:103)
>       at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.<init>(TungstenAggregationIterator.scala:483)
>       at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:95)
>       at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:86)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73)
>       at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
>       at org.apache.spark.scheduler.Task.run(Task.scala:89)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:213)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> 15/12/05 01:20:57 ERROR SparkUncaughtExceptionHandler: Uncaught exception in 
> thread Thread[Executor task launch worker-4,5,main]
> java.lang.OutOfMemoryError: Unable to acquire 262144 bytes of memory, got 0
>       at 
> org.apache.spark.memory.MemoryConsumer.allocateArray(MemoryConsumer.java:91)
>       at 
> org.apache.spark.unsafe.map.BytesToBytesMap.allocate(BytesToBytesMap.java:735)
>       at 
> org.apache.spark.unsafe.map.BytesToBytesMap.<init>(BytesToBytesMap.java:197)
>       at 
> org.apache.spark.unsafe.map.BytesToBytesMap.<init>(BytesToBytesMap.java:212)
>       at 
> org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap.<init>(UnsafeFixedWidthAggregationMap.java:103)
>       at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.<init>(TungstenAggregationIterator.scala:483)
>       at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:95)
>       at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:86)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73)
>       at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
>       at org.apache.spark.scheduler.Task.run(Task.scala:89)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:213)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> 15/12/05 01:20:57 INFO DiskBlockManager: Shutdown hook called
> 15/12/05 01:20:57 INFO GenerateMutableProjection: Code generated in 21.666344 
> ms
> 15/12/05 01:20:57 DEBUG KeepAliveThread: KeepAliveThread received command: 
> Shutdown
> 15/12/05 01:20:57 ERROR SparkUncaughtExceptionHandler: [Container in 
> shutdown] Uncaught exception in thread Thread[Executor task launch 
> worker-6,5,main]
> java.lang.OutOfMemoryError: Unable to acquire 262144 bytes of memory, got 0
>       at 
> org.apache.spark.memory.MemoryConsumer.allocateArray(MemoryConsumer.java:91)
>       at 
> org.apache.spark.unsafe.map.BytesToBytesMap.allocate(BytesToBytesMap.java:735)
>       at 
> org.apache.spark.unsafe.map.BytesToBytesMap.<init>(BytesToBytesMap.java:197)
>       at 
> org.apache.spark.unsafe.map.BytesToBytesMap.<init>(BytesToBytesMap.java:212)
>       at 
> org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap.<init>(UnsafeFixedWidthAggregationMap.java:103)
>       at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.<init>(TungstenAggregationIterator.scala:483)
>       at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:95)
>       at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:86)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73)
>       at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
>       at org.apache.spark.scheduler.Task.run(Task.scala:89)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:213)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> 15/12/05 01:20:57 ERROR SparkUncaughtExceptionHandler: [Container in 
> shutdown] Uncaught exception in thread Thread[Executor task launch 
> worker-7,5,main]
> java.lang.OutOfMemoryError: Unable to acquire 262144 bytes of memory, got 0
>       at 
> org.apache.spark.memory.MemoryConsumer.allocateArray(MemoryConsumer.java:91)
>       at 
> org.apache.spark.unsafe.map.BytesToBytesMap.allocate(BytesToBytesMap.java:735)
>       at 
> org.apache.spark.unsafe.map.BytesToBytesMap.<init>(BytesToBytesMap.java:197)
>       at 
> org.apache.spark.unsafe.map.BytesToBytesMap.<init>(BytesToBytesMap.java:212)
>       at 
> org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap.<init>(UnsafeFixedWidthAggregationMap.java:103)
>       at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.<init>(TungstenAggregationIterator.scala:483)
>       at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:95)
>       at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:86)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
>       at 
> org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73)
>       at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
>       at org.apache.spark.scheduler.Task.run(Task.scala:89)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:213)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> 15/12/05 01:20:57 INFO KeepAliveThread: KeepAlive thread has been shutdown 
> successfully
> 15/12/05 01:20:57 WARN TaskMemoryManager: leak 28.1 MB memory from 
> org.apache.spark.unsafe.map.BytesToBytesMap@6feafdad
> 15/12/05 01:20:57 ERROR Executor: Managed memory leak detected; size = 
> 29425664 bytes, TID = 120
> 15/12/05 01:20:57 ERROR Executor: Exception in task 1.0 in stage 5.0 (TID 120)
> java.io.FileNotFoundException: 
> /local_disk/spark-1ebb23ad-e3a1-4af2-b3d0-58a70ceed7ec/executor-ca2c389d-8b67-487f-b175-b867282bf0a3/blockmgr-deda3833-d86c-4850-aa4f-64c26ebfbc4f/08/temp_shuffle_8b5df98d-701c-4ef3-98cc-9e4731fe4a68
>  (No such file or directory)
>       at java.io.FileOutputStream.open0(Native Method)
>       at java.io.FileOutputStream.open(FileOutputStream.java:270)
>       at java.io.FileOutputStream.<init>(FileOutputStream.java:213)
>       at 
> org.apache.spark.storage.DiskBlockObjectWriter.open(DiskBlockObjectWriter.scala:88)
>       at 
> org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:140)
>       at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73)
>       at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
>       at org.apache.spark.scheduler.Task.run(Task.scala:89)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:213)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> 15/12/05 01:20:57 INFO ShutdownHookManager: Shutdown hook called
> {code}
> The query plan was like 
> {code}
> TungstenAggregate4
> +- TungstenExchange2
>    +- TungstenAggregate3
>       +- TungstenAggregate2
>          +- TungstenExchange1
>             +- TungstenAggregate1
>                +- Project 
>                   +- InMemoryColumnarTableScan
> {code}
> OOM happened in the stage having TungstenAggregate2 and TungstenAggregate3.



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

Reply via email to