You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@spark.apache.org by Ted Yu <yu...@gmail.com> on 2015/11/02 16:42:52 UTC

Re: test failed due to OOME

Looks like SparkListenerSuite doesn't OOM on QA runs compared to Jenkins
builds.

I wonder if this is due to difference between machines running QA tests vs
machines running Jenkins builds.

On Fri, Oct 30, 2015 at 1:19 PM, Ted Yu <yu...@gmail.com> wrote:

> I noticed that the SparkContext created in each sub-test is not stopped
> upon finishing sub-test.
>
> Would stopping each SparkContext make a difference in terms of heap memory
> consumption ?
>
> Cheers
>
> On Fri, Oct 30, 2015 at 12:04 PM, Mridul Muralidharan <mr...@gmail.com>
> wrote:
>
>> It is giving OOM at 32GB ? Something looks wrong with that ... that is
>> already on the higher side.
>>
>> Regards,
>> Mridul
>>
>>
>> On Fri, Oct 30, 2015 at 11:28 AM, shane knapp <sk...@berkeley.edu>
>> wrote:
>> > here's the current heap settings on our workers:
>> > InitialHeapSize == 2.1G
>> > MaxHeapSize == 32G
>> >
>> > system ram:  128G
>> >
>> > we can bump it pretty easily...  it's just a matter of deciding if we
>> > want to do this globally (super easy, but will affect ALL maven builds
>> > on our system -- not just spark) or on a per-job basis (this doesn't
>> > scale that well).
>> >
>> > thoughts?
>> >
>> > On Fri, Oct 30, 2015 at 9:47 AM, Ted Yu <yu...@gmail.com> wrote:
>> >> This happened recently on Jenkins:
>> >>
>> >>
>> https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-Maven-with-YARN/HADOOP_PROFILE=hadoop-2.3,label=spark-test/3964/console
>> >>
>> >> On Sun, Oct 18, 2015 at 7:54 AM, Ted Yu <yu...@gmail.com> wrote:
>> >>>
>> >>> From
>> >>>
>> https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-Maven-with-YARN/HADOOP_PROFILE=hadoop-2.4,label=spark-test/3846/console
>> >>> :
>> >>>
>> >>> SparkListenerSuite:
>> >>> - basic creation and shutdown of LiveListenerBus
>> >>> - bus.stop() waits for the event queue to completely drain
>> >>> - basic creation of StageInfo
>> >>> - basic creation of StageInfo with shuffle
>> >>> - StageInfo with fewer tasks than partitions
>> >>> - local metrics
>> >>> - onTaskGettingResult() called when result fetched remotely ***
>> FAILED ***
>> >>>   org.apache.spark.SparkException: Job aborted due to stage failure:
>> Task
>> >>> 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in
>> stage
>> >>> 0.0 (TID 0, localhost): java.lang.OutOfMemoryError: Java heap space
>> >>>      at java.util.Arrays.copyOf(Arrays.java:2271)
>> >>>      at
>> java.io.ByteArrayOutputStream.grow(ByteArrayOutputStream.java:113)
>> >>>      at
>> >>>
>> java.io.ByteArrayOutputStream.ensureCapacity(ByteArrayOutputStream.java:93)
>> >>>      at
>> java.io.ByteArrayOutputStream.write(ByteArrayOutputStream.java:140)
>> >>>      at
>> >>>
>> java.io.ObjectOutputStream$BlockDataOutputStream.write(ObjectOutputStream.java:1852)
>> >>>      at java.io.ObjectOutputStream.write(ObjectOutputStream.java:708)
>> >>>      at org.apache.spark.util.Utils$.writeByteBuffer(Utils.scala:182)
>> >>>      at
>> >>>
>> org.apache.spark.scheduler.DirectTaskResult$$anonfun$writeExternal$1.apply$mcV$sp(TaskResult.scala:52)
>> >>>      at
>> org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1160)
>> >>>      at
>> >>>
>> org.apache.spark.scheduler.DirectTaskResult.writeExternal(TaskResult.scala:49)
>> >>>      at
>> >>>
>> java.io.ObjectOutputStream.writeExternalData(ObjectOutputStream.java:1458)
>> >>>      at
>> >>>
>> java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1429)
>> >>>      at
>> java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177)
>> >>>      at
>> java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:347)
>> >>>      at
>> >>>
>> org.apache.spark.serializer.JavaSerializationStream.writeObject(JavaSerializer.scala:44)
>> >>>      at
>> >>>
>> org.apache.spark.serializer.JavaSerializerInstance.serialize(JavaSerializer.scala:101)
>> >>>      at
>> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:256)
>> >>>      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)
>> >>>
>> >>>
>> >>> Should more heap be given to test suite ?
>> >>>
>> >>>
>> >>> Cheers
>> >>
>> >>
>> >
>> > ---------------------------------------------------------------------
>> > To unsubscribe, e-mail: dev-unsubscribe@spark.apache.org
>> > For additional commands, e-mail: dev-help@spark.apache.org
>> >
>>
>
>

Re: test failed due to OOME

Posted by Ted Yu <yu...@gmail.com>.
I have a PR which tries to address this issue:
https://github.com/apache/spark/pull/9384

Comment is welcome.

On Mon, Nov 2, 2015 at 9:53 AM, Patrick Wendell <pw...@gmail.com> wrote:

> I believe this is some bug in our tests. For some reason we are using way
> more memory than necessary. We'll probably need to log into Jenkins and
> heap dump some running tests and figure out what is going on.
>
> On Mon, Nov 2, 2015 at 7:42 AM, Ted Yu <yu...@gmail.com> wrote:
>
>> Looks like SparkListenerSuite doesn't OOM on QA runs compared to Jenkins
>> builds.
>>
>> I wonder if this is due to difference between machines running QA tests
>> vs machines running Jenkins builds.
>>
>> On Fri, Oct 30, 2015 at 1:19 PM, Ted Yu <yu...@gmail.com> wrote:
>>
>>> I noticed that the SparkContext created in each sub-test is not stopped
>>> upon finishing sub-test.
>>>
>>> Would stopping each SparkContext make a difference in terms of heap
>>> memory consumption ?
>>>
>>> Cheers
>>>
>>> On Fri, Oct 30, 2015 at 12:04 PM, Mridul Muralidharan <mr...@gmail.com>
>>> wrote:
>>>
>>>> It is giving OOM at 32GB ? Something looks wrong with that ... that is
>>>> already on the higher side.
>>>>
>>>> Regards,
>>>> Mridul
>>>>
>>>>
>>>> On Fri, Oct 30, 2015 at 11:28 AM, shane knapp <sk...@berkeley.edu>
>>>> wrote:
>>>> > here's the current heap settings on our workers:
>>>> > InitialHeapSize == 2.1G
>>>> > MaxHeapSize == 32G
>>>> >
>>>> > system ram:  128G
>>>> >
>>>> > we can bump it pretty easily...  it's just a matter of deciding if we
>>>> > want to do this globally (super easy, but will affect ALL maven builds
>>>> > on our system -- not just spark) or on a per-job basis (this doesn't
>>>> > scale that well).
>>>> >
>>>> > thoughts?
>>>> >
>>>> > On Fri, Oct 30, 2015 at 9:47 AM, Ted Yu <yu...@gmail.com> wrote:
>>>> >> This happened recently on Jenkins:
>>>> >>
>>>> >>
>>>> https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-Maven-with-YARN/HADOOP_PROFILE=hadoop-2.3,label=spark-test/3964/console
>>>> >>
>>>> >> On Sun, Oct 18, 2015 at 7:54 AM, Ted Yu <yu...@gmail.com> wrote:
>>>> >>>
>>>> >>> From
>>>> >>>
>>>> https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-Maven-with-YARN/HADOOP_PROFILE=hadoop-2.4,label=spark-test/3846/console
>>>> >>> :
>>>> >>>
>>>> >>> SparkListenerSuite:
>>>> >>> - basic creation and shutdown of LiveListenerBus
>>>> >>> - bus.stop() waits for the event queue to completely drain
>>>> >>> - basic creation of StageInfo
>>>> >>> - basic creation of StageInfo with shuffle
>>>> >>> - StageInfo with fewer tasks than partitions
>>>> >>> - local metrics
>>>> >>> - onTaskGettingResult() called when result fetched remotely ***
>>>> FAILED ***
>>>> >>>   org.apache.spark.SparkException: Job aborted due to stage
>>>> failure: Task
>>>> >>> 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0
>>>> in stage
>>>> >>> 0.0 (TID 0, localhost): java.lang.OutOfMemoryError: Java heap space
>>>> >>>      at java.util.Arrays.copyOf(Arrays.java:2271)
>>>> >>>      at
>>>> java.io.ByteArrayOutputStream.grow(ByteArrayOutputStream.java:113)
>>>> >>>      at
>>>> >>>
>>>> java.io.ByteArrayOutputStream.ensureCapacity(ByteArrayOutputStream.java:93)
>>>> >>>      at
>>>> java.io.ByteArrayOutputStream.write(ByteArrayOutputStream.java:140)
>>>> >>>      at
>>>> >>>
>>>> java.io.ObjectOutputStream$BlockDataOutputStream.write(ObjectOutputStream.java:1852)
>>>> >>>      at
>>>> java.io.ObjectOutputStream.write(ObjectOutputStream.java:708)
>>>> >>>      at
>>>> org.apache.spark.util.Utils$.writeByteBuffer(Utils.scala:182)
>>>> >>>      at
>>>> >>>
>>>> org.apache.spark.scheduler.DirectTaskResult$$anonfun$writeExternal$1.apply$mcV$sp(TaskResult.scala:52)
>>>> >>>      at
>>>> org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1160)
>>>> >>>      at
>>>> >>>
>>>> org.apache.spark.scheduler.DirectTaskResult.writeExternal(TaskResult.scala:49)
>>>> >>>      at
>>>> >>>
>>>> java.io.ObjectOutputStream.writeExternalData(ObjectOutputStream.java:1458)
>>>> >>>      at
>>>> >>>
>>>> java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1429)
>>>> >>>      at
>>>> java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177)
>>>> >>>      at
>>>> java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:347)
>>>> >>>      at
>>>> >>>
>>>> org.apache.spark.serializer.JavaSerializationStream.writeObject(JavaSerializer.scala:44)
>>>> >>>      at
>>>> >>>
>>>> org.apache.spark.serializer.JavaSerializerInstance.serialize(JavaSerializer.scala:101)
>>>> >>>      at
>>>> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:256)
>>>> >>>      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)
>>>> >>>
>>>> >>>
>>>> >>> Should more heap be given to test suite ?
>>>> >>>
>>>> >>>
>>>> >>> Cheers
>>>> >>
>>>> >>
>>>> >
>>>> > ---------------------------------------------------------------------
>>>> > To unsubscribe, e-mail: dev-unsubscribe@spark.apache.org
>>>> > For additional commands, e-mail: dev-help@spark.apache.org
>>>> >
>>>>
>>>
>>>
>>
>

Re: test failed due to OOME

Posted by Patrick Wendell <pw...@gmail.com>.
I believe this is some bug in our tests. For some reason we are using way
more memory than necessary. We'll probably need to log into Jenkins and
heap dump some running tests and figure out what is going on.

On Mon, Nov 2, 2015 at 7:42 AM, Ted Yu <yu...@gmail.com> wrote:

> Looks like SparkListenerSuite doesn't OOM on QA runs compared to Jenkins
> builds.
>
> I wonder if this is due to difference between machines running QA tests vs
> machines running Jenkins builds.
>
> On Fri, Oct 30, 2015 at 1:19 PM, Ted Yu <yu...@gmail.com> wrote:
>
>> I noticed that the SparkContext created in each sub-test is not stopped
>> upon finishing sub-test.
>>
>> Would stopping each SparkContext make a difference in terms of heap
>> memory consumption ?
>>
>> Cheers
>>
>> On Fri, Oct 30, 2015 at 12:04 PM, Mridul Muralidharan <mr...@gmail.com>
>> wrote:
>>
>>> It is giving OOM at 32GB ? Something looks wrong with that ... that is
>>> already on the higher side.
>>>
>>> Regards,
>>> Mridul
>>>
>>>
>>> On Fri, Oct 30, 2015 at 11:28 AM, shane knapp <sk...@berkeley.edu>
>>> wrote:
>>> > here's the current heap settings on our workers:
>>> > InitialHeapSize == 2.1G
>>> > MaxHeapSize == 32G
>>> >
>>> > system ram:  128G
>>> >
>>> > we can bump it pretty easily...  it's just a matter of deciding if we
>>> > want to do this globally (super easy, but will affect ALL maven builds
>>> > on our system -- not just spark) or on a per-job basis (this doesn't
>>> > scale that well).
>>> >
>>> > thoughts?
>>> >
>>> > On Fri, Oct 30, 2015 at 9:47 AM, Ted Yu <yu...@gmail.com> wrote:
>>> >> This happened recently on Jenkins:
>>> >>
>>> >>
>>> https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-Maven-with-YARN/HADOOP_PROFILE=hadoop-2.3,label=spark-test/3964/console
>>> >>
>>> >> On Sun, Oct 18, 2015 at 7:54 AM, Ted Yu <yu...@gmail.com> wrote:
>>> >>>
>>> >>> From
>>> >>>
>>> https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-Maven-with-YARN/HADOOP_PROFILE=hadoop-2.4,label=spark-test/3846/console
>>> >>> :
>>> >>>
>>> >>> SparkListenerSuite:
>>> >>> - basic creation and shutdown of LiveListenerBus
>>> >>> - bus.stop() waits for the event queue to completely drain
>>> >>> - basic creation of StageInfo
>>> >>> - basic creation of StageInfo with shuffle
>>> >>> - StageInfo with fewer tasks than partitions
>>> >>> - local metrics
>>> >>> - onTaskGettingResult() called when result fetched remotely ***
>>> FAILED ***
>>> >>>   org.apache.spark.SparkException: Job aborted due to stage failure:
>>> Task
>>> >>> 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in
>>> stage
>>> >>> 0.0 (TID 0, localhost): java.lang.OutOfMemoryError: Java heap space
>>> >>>      at java.util.Arrays.copyOf(Arrays.java:2271)
>>> >>>      at
>>> java.io.ByteArrayOutputStream.grow(ByteArrayOutputStream.java:113)
>>> >>>      at
>>> >>>
>>> java.io.ByteArrayOutputStream.ensureCapacity(ByteArrayOutputStream.java:93)
>>> >>>      at
>>> java.io.ByteArrayOutputStream.write(ByteArrayOutputStream.java:140)
>>> >>>      at
>>> >>>
>>> java.io.ObjectOutputStream$BlockDataOutputStream.write(ObjectOutputStream.java:1852)
>>> >>>      at java.io.ObjectOutputStream.write(ObjectOutputStream.java:708)
>>> >>>      at org.apache.spark.util.Utils$.writeByteBuffer(Utils.scala:182)
>>> >>>      at
>>> >>>
>>> org.apache.spark.scheduler.DirectTaskResult$$anonfun$writeExternal$1.apply$mcV$sp(TaskResult.scala:52)
>>> >>>      at
>>> org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1160)
>>> >>>      at
>>> >>>
>>> org.apache.spark.scheduler.DirectTaskResult.writeExternal(TaskResult.scala:49)
>>> >>>      at
>>> >>>
>>> java.io.ObjectOutputStream.writeExternalData(ObjectOutputStream.java:1458)
>>> >>>      at
>>> >>>
>>> java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1429)
>>> >>>      at
>>> java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177)
>>> >>>      at
>>> java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:347)
>>> >>>      at
>>> >>>
>>> org.apache.spark.serializer.JavaSerializationStream.writeObject(JavaSerializer.scala:44)
>>> >>>      at
>>> >>>
>>> org.apache.spark.serializer.JavaSerializerInstance.serialize(JavaSerializer.scala:101)
>>> >>>      at
>>> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:256)
>>> >>>      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)
>>> >>>
>>> >>>
>>> >>> Should more heap be given to test suite ?
>>> >>>
>>> >>>
>>> >>> Cheers
>>> >>
>>> >>
>>> >
>>> > ---------------------------------------------------------------------
>>> > To unsubscribe, e-mail: dev-unsubscribe@spark.apache.org
>>> > For additional commands, e-mail: dev-help@spark.apache.org
>>> >
>>>
>>
>>
>