Re: java.lang.OutOfMemoryError: Unable to acquire bytes of memory

2016-04-14 Thread Nezih Yigitbasi
Thanks Imran. I will give it a shot when I have some time.

Nezih

On Thu, Apr 14, 2016 at 9:25 AM Imran Rashid  wrote:

> Hi Nezih,
>
> I just reported a somewhat similar issue, and I have a potential fix --
> SPARK-14560, looks like you are already watching it :).  You can try out
> that patch, you have to explicitly enable the change in behavior with
> "spark.shuffle.spillAfterRead=true".  Honestly, I don't think these issues
> are the same, as I've always seen that case lead to acquiring 0 bytes,
> while in your case you are requesting GBs and getting something pretty
> close, so my hunch is that it is different ... but might be worth a shot to
> see if it is the issue.
>
> Turning on debug logging for TaskMemoryManager might help track the root
> cause -- you'll get information on which consumers are using memory and
> when there are spill attempts.  (Note that even if the patch I have for
> SPARK-14560 doesn't fix your issue, it might still make those debug logs a
> bit more clear, since it'll report memory used by Spillables.)
>
> Imran
>
> On Mon, Apr 4, 2016 at 10:52 PM, Nezih Yigitbasi <
> nyigitb...@netflix.com.invalid> wrote:
>
>> Nope, I didn't have a chance to track the root cause, and IIRC we didn't
>> observe it when dyn. alloc. is off.
>>
>> On Mon, Apr 4, 2016 at 6:16 PM Reynold Xin  wrote:
>>
>>> BTW do you still see this when dynamic allocation is off?
>>>
>>> On Mon, Apr 4, 2016 at 6:16 PM, Reynold Xin  wrote:
>>>
 Nezih,

 Have you had a chance to figure out why this is happening?


 On Tue, Mar 22, 2016 at 1:32 AM, james  wrote:

> I guess different workload cause diff result ?
>
>
>
> --
> View this message in context:
> http://apache-spark-developers-list.1001551.n3.nabble.com/java-lang-OutOfMemoryError-Unable-to-acquire-bytes-of-memory-tp16773p16789.html
> Sent from the Apache Spark Developers List mailing list archive at
> Nabble.com.
>
> -
> To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org
> For additional commands, e-mail: dev-h...@spark.apache.org
>
>

>>>
>


Re: java.lang.OutOfMemoryError: Unable to acquire bytes of memory

2016-04-14 Thread Imran Rashid
Hi Nezih,

I just reported a somewhat similar issue, and I have a potential fix --
SPARK-14560, looks like you are already watching it :).  You can try out
that patch, you have to explicitly enable the change in behavior with
"spark.shuffle.spillAfterRead=true".  Honestly, I don't think these issues
are the same, as I've always seen that case lead to acquiring 0 bytes,
while in your case you are requesting GBs and getting something pretty
close, so my hunch is that it is different ... but might be worth a shot to
see if it is the issue.

Turning on debug logging for TaskMemoryManager might help track the root
cause -- you'll get information on which consumers are using memory and
when there are spill attempts.  (Note that even if the patch I have for
SPARK-14560 doesn't fix your issue, it might still make those debug logs a
bit more clear, since it'll report memory used by Spillables.)

Imran

On Mon, Apr 4, 2016 at 10:52 PM, Nezih Yigitbasi <
nyigitb...@netflix.com.invalid> wrote:

> Nope, I didn't have a chance to track the root cause, and IIRC we didn't
> observe it when dyn. alloc. is off.
>
> On Mon, Apr 4, 2016 at 6:16 PM Reynold Xin  wrote:
>
>> BTW do you still see this when dynamic allocation is off?
>>
>> On Mon, Apr 4, 2016 at 6:16 PM, Reynold Xin  wrote:
>>
>>> Nezih,
>>>
>>> Have you had a chance to figure out why this is happening?
>>>
>>>
>>> On Tue, Mar 22, 2016 at 1:32 AM, james  wrote:
>>>
 I guess different workload cause diff result ?



 --
 View this message in context:
 http://apache-spark-developers-list.1001551.n3.nabble.com/java-lang-OutOfMemoryError-Unable-to-acquire-bytes-of-memory-tp16773p16789.html
 Sent from the Apache Spark Developers List mailing list archive at
 Nabble.com.

 -
 To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org
 For additional commands, e-mail: dev-h...@spark.apache.org


>>>
>>


Re: java.lang.OutOfMemoryError: Unable to acquire bytes of memory

2016-04-04 Thread Nezih Yigitbasi
Nope, I didn't have a chance to track the root cause, and IIRC we didn't
observe it when dyn. alloc. is off.

On Mon, Apr 4, 2016 at 6:16 PM Reynold Xin  wrote:

> BTW do you still see this when dynamic allocation is off?
>
> On Mon, Apr 4, 2016 at 6:16 PM, Reynold Xin  wrote:
>
>> Nezih,
>>
>> Have you had a chance to figure out why this is happening?
>>
>>
>> On Tue, Mar 22, 2016 at 1:32 AM, james  wrote:
>>
>>> I guess different workload cause diff result ?
>>>
>>>
>>>
>>> --
>>> View this message in context:
>>> http://apache-spark-developers-list.1001551.n3.nabble.com/java-lang-OutOfMemoryError-Unable-to-acquire-bytes-of-memory-tp16773p16789.html
>>> Sent from the Apache Spark Developers List mailing list archive at
>>> Nabble.com.
>>>
>>> -
>>> To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org
>>> For additional commands, e-mail: dev-h...@spark.apache.org
>>>
>>>
>>
>


Re: java.lang.OutOfMemoryError: Unable to acquire bytes of memory

2016-04-04 Thread Reynold Xin
Nezih,

Have you had a chance to figure out why this is happening?


On Tue, Mar 22, 2016 at 1:32 AM, james  wrote:

> I guess different workload cause diff result ?
>
>
>
> --
> View this message in context:
> http://apache-spark-developers-list.1001551.n3.nabble.com/java-lang-OutOfMemoryError-Unable-to-acquire-bytes-of-memory-tp16773p16789.html
> Sent from the Apache Spark Developers List mailing list archive at
> Nabble.com.
>
> -
> To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org
> For additional commands, e-mail: dev-h...@spark.apache.org
>
>


Re: java.lang.OutOfMemoryError: Unable to acquire bytes of memory

2016-04-04 Thread Reynold Xin
BTW do you still see this when dynamic allocation is off?

On Mon, Apr 4, 2016 at 6:16 PM, Reynold Xin  wrote:

> Nezih,
>
> Have you had a chance to figure out why this is happening?
>
>
> On Tue, Mar 22, 2016 at 1:32 AM, james  wrote:
>
>> I guess different workload cause diff result ?
>>
>>
>>
>> --
>> View this message in context:
>> http://apache-spark-developers-list.1001551.n3.nabble.com/java-lang-OutOfMemoryError-Unable-to-acquire-bytes-of-memory-tp16773p16789.html
>> Sent from the Apache Spark Developers List mailing list archive at
>> Nabble.com.
>>
>> -
>> To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org
>> For additional commands, e-mail: dev-h...@spark.apache.org
>>
>>
>


Re: java.lang.OutOfMemoryError: Unable to acquire bytes of memory

2016-03-22 Thread james
I guess different workload cause diff result ?



--
View this message in context: 
http://apache-spark-developers-list.1001551.n3.nabble.com/java-lang-OutOfMemoryError-Unable-to-acquire-bytes-of-memory-tp16773p16789.html
Sent from the Apache Spark Developers List mailing list archive at Nabble.com.

-
To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org
For additional commands, e-mail: dev-h...@spark.apache.org



Re: java.lang.OutOfMemoryError: Unable to acquire bytes of memory

2016-03-22 Thread Nezih Yigitbasi
Interesting. After experimenting with various parameters increasing
spark.sql.shuffle.partitions and decreasing spark.buffer.pageSize helped my
job go through. BTW I will be happy to help getting this issue fixed.

Nezih

On Tue, Mar 22, 2016 at 1:07 AM james  wrote:

Hi,
> I also found 'Unable to acquire memory' issue using Spark 1.6.1 with
> Dynamic
> allocation on YARN. My case happened with setting
> spark.sql.shuffle.partitions larger than 200. From error stack, it has a
> diff with issue reported by Nezih and not sure if these has same root
> cause.
>
> Thanks
> James
>
> 16/03/17 16:02:11 INFO spark.MapOutputTrackerMaster: Size of output
> statuses
> for shuffle 0 is 1912805 bytes
> 16/03/17 16:02:12 INFO spark.MapOutputTrackerMasterEndpoint: Asked to send
> map output locations for shuffle 1 to hw-node3:55062
> 16/03/17 16:02:12 INFO spark.MapOutputTrackerMaster: Size of output
> statuses
> for shuffle 0 is 1912805 bytes
> 16/03/17 16:02:16 INFO scheduler.TaskSetManager: Starting task 280.0 in
> stage 153.0 (TID 9390, hw-node5, partition 280,PROCESS_LOCAL, 2432 bytes)
> 16/03/17 16:02:16 WARN scheduler.TaskSetManager: Lost task 170.0 in stage
> 153.0 (TID 9280, hw-node5): java.lang.OutOfMemoryError: Unable to acquire
> 1073741824 bytes of memory, got 1060110796
> at
>
> org.apache.spark.memory.MemoryConsumer.allocateArray(MemoryConsumer.java:91)
> at
>
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.growPointerArrayIfNecessary(UnsafeExternalSorter.java:295)
> at
>
> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertRecord(UnsafeExternalSorter.java:330)
> at
>
> org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:91)
> at
>
> org.apache.spark.sql.execution.UnsafeExternalRowSorter.sort(UnsafeExternalRowSorter.java:168)
> at
> org.apache.spark.sql.execution.Sort$anonfun$1.apply(Sort.scala:90)
> at
> org.apache.spark.sql.execution.Sort$anonfun$1.apply(Sort.scala:64)
> at
>
> org.apache.spark.rdd.RDD$anonfun$mapPartitionsInternal$1$anonfun$apply$21.apply(RDD.scala:728)
> at
>
> org.apache.spark.rdd.RDD$anonfun$mapPartitionsInternal$1$anonfun$apply$21.apply(RDD.scala:728)
> 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.ZippedPartitionsRDD2.compute(ZippedPartitionsRDD.scala:88)
> 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.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:214)
> 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)
>
>
>
> --
> View this message in context:
> http://apache-spark-developers-list.1001551.n3.nabble.com/java-lang-OutOfMemoryError-Unable-to-acquire-bytes-of-memory-tp16773p16787.html
> Sent from the Apache Spark Developers List mailing list archive at
> Nabble.com.
>
> -
> To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org
> For additional commands, e-mail: dev-h...@spark.apache.org
>
> ​


Re: java.lang.OutOfMemoryError: Unable to acquire bytes of memory

2016-03-22 Thread james
Hi,
I also found 'Unable to acquire memory' issue using Spark 1.6.1 with Dynamic
allocation on YARN. My case happened with setting
spark.sql.shuffle.partitions larger than 200. From error stack, it has a
diff with issue reported by Nezih and not sure if these has same root cause.

Thanks 
James

16/03/17 16:02:11 INFO spark.MapOutputTrackerMaster: Size of output statuses
for shuffle 0 is 1912805 bytes
16/03/17 16:02:12 INFO spark.MapOutputTrackerMasterEndpoint: Asked to send
map output locations for shuffle 1 to hw-node3:55062
16/03/17 16:02:12 INFO spark.MapOutputTrackerMaster: Size of output statuses
for shuffle 0 is 1912805 bytes
16/03/17 16:02:16 INFO scheduler.TaskSetManager: Starting task 280.0 in
stage 153.0 (TID 9390, hw-node5, partition 280,PROCESS_LOCAL, 2432 bytes)
16/03/17 16:02:16 WARN scheduler.TaskSetManager: Lost task 170.0 in stage
153.0 (TID 9280, hw-node5): java.lang.OutOfMemoryError: Unable to acquire
1073741824 bytes of memory, got 1060110796
at
org.apache.spark.memory.MemoryConsumer.allocateArray(MemoryConsumer.java:91)
at
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.growPointerArrayIfNecessary(UnsafeExternalSorter.java:295)
at
org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertRecord(UnsafeExternalSorter.java:330)
at
org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:91)
at
org.apache.spark.sql.execution.UnsafeExternalRowSorter.sort(UnsafeExternalRowSorter.java:168)
at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:90)
at org.apache.spark.sql.execution.Sort$$anonfun$1.apply(Sort.scala:64)
at
org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728)
at
org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728)
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.ZippedPartitionsRDD2.compute(ZippedPartitionsRDD.scala:88)
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.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:214)
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)



--
View this message in context: 
http://apache-spark-developers-list.1001551.n3.nabble.com/java-lang-OutOfMemoryError-Unable-to-acquire-bytes-of-memory-tp16773p16787.html
Sent from the Apache Spark Developers List mailing list archive at Nabble.com.

-
To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org
For additional commands, e-mail: dev-h...@spark.apache.org



Re: java.lang.OutOfMemoryError: Unable to acquire bytes of memory

2016-03-21 Thread Nezih Yigitbasi
Andrew, thanks for the suggestion, but unfortunately it didn't work --
still getting the same exception.
On Mon, Mar 21, 2016 at 10:32 AM Andrew Or  wrote:

> @Nezih, can you try again after setting `spark.memory.useLegacyMode` to
> true? Can you still reproduce the OOM that way?
>
> 2016-03-21 10:29 GMT-07:00 Nezih Yigitbasi  >:
>
>> Hi Spark devs,
>> I am using 1.6.0 with dynamic allocation on yarn. I am trying to run a
>> relatively big application with 10s of jobs and 100K+ tasks and my app
>> fails with the exception below. The closest jira issue I could find is
>> SPARK-11293 , which
>> is a critical bug that has been open for a long time. There are other
>> similar jira issues (all fixed): SPARK-10474
>> , SPARK-10733
>> , SPARK-10309
>> , SPARK-10379
>> .
>>
>> Any workarounds to this issue or any plans to fix it?
>>
>> Thanks a lot,
>> Nezih
>>
>> 16/03/19 05:12:09 INFO memory.TaskMemoryManager: Memory used in task 
>> 4687016/03/19 05:12:09 INFO memory.TaskMemoryManager: Acquired by 
>> org.apache.spark.shuffle.sort.ShuffleExternalSorter@1c36f801: 32.0 
>> KB16/03/19 05:12:09 INFO memory.TaskMemoryManager: 1512915599 bytes of 
>> memory were used by task 46870 but are not associated with specific 
>> consumers16/03/19 05:12:09 INFO memory.TaskMemoryManager: 1512948367 bytes 
>> of memory are used for execution and 156978343 bytes of memory are used for 
>> storage16/03/19 05:12:09 ERROR executor.Executor: Managed memory leak 
>> detected; size = 1512915599 bytes, TID = 4687016/03/19 05:12:09 ERROR 
>> executor.Executor: Exception in task 77.0 in stage 273.0 (TID 46870)
>> java.lang.OutOfMemoryError: Unable to acquire 128 bytes of memory, got 0
>> at 
>> org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:120)
>> at 
>> org.apache.spark.shuffle.sort.ShuffleExternalSorter.acquireNewPageIfNecessary(ShuffleExternalSorter.java:354)
>> at 
>> org.apache.spark.shuffle.sort.ShuffleExternalSorter.insertRecord(ShuffleExternalSorter.java:375)
>> at 
>> org.apache.spark.shuffle.sort.UnsafeShuffleWriter.insertRecordIntoSorter(UnsafeShuffleWriter.java:237)
>> at 
>> org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:164)
>> 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)16/03/19 05:12:09 ERROR 
>> util.SparkUncaughtExceptionHandler: Uncaught exception in thread 
>> Thread[Executor task launch worker-8,5,main]
>> java.lang.OutOfMemoryError: Unable to acquire 128 bytes of memory, got 0
>> at 
>> org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:120)
>> at 
>> org.apache.spark.shuffle.sort.ShuffleExternalSorter.acquireNewPageIfNecessary(ShuffleExternalSorter.java:354)
>> at 
>> org.apache.spark.shuffle.sort.ShuffleExternalSorter.insertRecord(ShuffleExternalSorter.java:375)
>> at 
>> org.apache.spark.shuffle.sort.UnsafeShuffleWriter.insertRecordIntoSorter(UnsafeShuffleWriter.java:237)
>> at 
>> org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:164)
>> 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)16/03/19 05:12:10 INFO 
>> storage.DiskBlockManager: Shutdown hook called16/03/19 05:12:10 INFO 
>> util.ShutdownHookManager: Shutdown hook called
>>
>> ​
>>
>
>


Re: java.lang.OutOfMemoryError: Unable to acquire bytes of memory

2016-03-21 Thread Andrew Or
@Nezih, can you try again after setting `spark.memory.useLegacyMode` to
true? Can you still reproduce the OOM that way?

2016-03-21 10:29 GMT-07:00 Nezih Yigitbasi :

> Hi Spark devs,
> I am using 1.6.0 with dynamic allocation on yarn. I am trying to run a
> relatively big application with 10s of jobs and 100K+ tasks and my app
> fails with the exception below. The closest jira issue I could find is
> SPARK-11293 , which is
> a critical bug that has been open for a long time. There are other similar
> jira issues (all fixed): SPARK-10474
> , SPARK-10733
> , SPARK-10309
> , SPARK-10379
> .
>
> Any workarounds to this issue or any plans to fix it?
>
> Thanks a lot,
> Nezih
>
> 16/03/19 05:12:09 INFO memory.TaskMemoryManager: Memory used in task 
> 4687016/03/19 05:12:09 INFO memory.TaskMemoryManager: Acquired by 
> org.apache.spark.shuffle.sort.ShuffleExternalSorter@1c36f801: 32.0 KB16/03/19 
> 05:12:09 INFO memory.TaskMemoryManager: 1512915599 bytes of memory were used 
> by task 46870 but are not associated with specific consumers16/03/19 05:12:09 
> INFO memory.TaskMemoryManager: 1512948367 bytes of memory are used for 
> execution and 156978343 bytes of memory are used for storage16/03/19 05:12:09 
> ERROR executor.Executor: Managed memory leak detected; size = 1512915599 
> bytes, TID = 4687016/03/19 05:12:09 ERROR executor.Executor: Exception in 
> task 77.0 in stage 273.0 (TID 46870)
> java.lang.OutOfMemoryError: Unable to acquire 128 bytes of memory, got 0
> at 
> org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:120)
> at 
> org.apache.spark.shuffle.sort.ShuffleExternalSorter.acquireNewPageIfNecessary(ShuffleExternalSorter.java:354)
> at 
> org.apache.spark.shuffle.sort.ShuffleExternalSorter.insertRecord(ShuffleExternalSorter.java:375)
> at 
> org.apache.spark.shuffle.sort.UnsafeShuffleWriter.insertRecordIntoSorter(UnsafeShuffleWriter.java:237)
> at 
> org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:164)
> 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)16/03/19 05:12:09 ERROR 
> util.SparkUncaughtExceptionHandler: Uncaught exception in thread 
> Thread[Executor task launch worker-8,5,main]
> java.lang.OutOfMemoryError: Unable to acquire 128 bytes of memory, got 0
> at 
> org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:120)
> at 
> org.apache.spark.shuffle.sort.ShuffleExternalSorter.acquireNewPageIfNecessary(ShuffleExternalSorter.java:354)
> at 
> org.apache.spark.shuffle.sort.ShuffleExternalSorter.insertRecord(ShuffleExternalSorter.java:375)
> at 
> org.apache.spark.shuffle.sort.UnsafeShuffleWriter.insertRecordIntoSorter(UnsafeShuffleWriter.java:237)
> at 
> org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:164)
> 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)16/03/19 05:12:10 INFO 
> storage.DiskBlockManager: Shutdown hook called16/03/19 05:12:10 INFO 
> util.ShutdownHookManager: Shutdown hook called
>
> ​
>


java.lang.OutOfMemoryError: Unable to acquire bytes of memory

2016-03-21 Thread Nezih Yigitbasi
Hi Spark devs,
I am using 1.6.0 with dynamic allocation on yarn. I am trying to run a
relatively big application with 10s of jobs and 100K+ tasks and my app
fails with the exception below. The closest jira issue I could find is
SPARK-11293 , which is a
critical bug that has been open for a long time. There are other similar
jira issues (all fixed): SPARK-10474
, SPARK-10733
, SPARK-10309
, SPARK-10379
.

Any workarounds to this issue or any plans to fix it?

Thanks a lot,
Nezih

16/03/19 05:12:09 INFO memory.TaskMemoryManager: Memory used in task
4687016/03/19 05:12:09 INFO memory.TaskMemoryManager: Acquired by
org.apache.spark.shuffle.sort.ShuffleExternalSorter@1c36f801: 32.0
KB16/03/19 05:12:09 INFO memory.TaskMemoryManager: 1512915599 bytes of
memory were used by task 46870 but are not associated with specific
consumers16/03/19 05:12:09 INFO memory.TaskMemoryManager: 1512948367
bytes of memory are used for execution and 156978343 bytes of memory
are used for storage16/03/19 05:12:09 ERROR executor.Executor: Managed
memory leak detected; size = 1512915599 bytes, TID = 4687016/03/19
05:12:09 ERROR executor.Executor: Exception in task 77.0 in stage
273.0 (TID 46870)
java.lang.OutOfMemoryError: Unable to acquire 128 bytes of memory, got 0
at 
org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:120)
at 
org.apache.spark.shuffle.sort.ShuffleExternalSorter.acquireNewPageIfNecessary(ShuffleExternalSorter.java:354)
at 
org.apache.spark.shuffle.sort.ShuffleExternalSorter.insertRecord(ShuffleExternalSorter.java:375)
at 
org.apache.spark.shuffle.sort.UnsafeShuffleWriter.insertRecordIntoSorter(UnsafeShuffleWriter.java:237)
at 
org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:164)
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)16/03/19 05:12:09 ERROR
util.SparkUncaughtExceptionHandler: Uncaught exception in thread
Thread[Executor task launch worker-8,5,main]
java.lang.OutOfMemoryError: Unable to acquire 128 bytes of memory, got 0
at 
org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:120)
at 
org.apache.spark.shuffle.sort.ShuffleExternalSorter.acquireNewPageIfNecessary(ShuffleExternalSorter.java:354)
at 
org.apache.spark.shuffle.sort.ShuffleExternalSorter.insertRecord(ShuffleExternalSorter.java:375)
at 
org.apache.spark.shuffle.sort.UnsafeShuffleWriter.insertRecordIntoSorter(UnsafeShuffleWriter.java:237)
at 
org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:164)
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)16/03/19 05:12:10 INFO
storage.DiskBlockManager: Shutdown hook called16/03/19 05:12:10 INFO
util.ShutdownHookManager: Shutdown hook called

​