If you end up with a really long dependency tree between RDDs (like 100+)
people have reported success with using the .checkpoint() method.  This
computes the RDD and then saves it, flattening the dependency tree.  It
turns out that having a really long RDD dependency graph causes
serialization sizes of tasks to go up, plus any failures causes a long
sequence of operations to regenerate the missing partition.

Maybe give that a shot and see if it helps?


On Fri, May 2, 2014 at 3:29 AM, Andrea Esposito <and1...@gmail.com> wrote:

> Sorry for the very late answer.
>
> I carefully follow what you have pointed out and i figure out that the
> structure used for each record was too big with many small objects.
> Changing it the memory usage drastically decrease.
>
> Despite that i'm still struggling with the behaviour of decreasing
> performance along supersteps. Now the memory footprint is much less than
> before and GC time is not noticeable anymore.
> I supposed that some RDDs are recomputed and watching carefully the stages
> there is evidence of that but i don't understand why it's happening.
>
> Recalling my usage pattern:
>
>> newRdd = oldRdd.map(myFun).persist(myStorageLevel)
>>
> newRdd.foreach(x => {}) // Force evaluation
>>
> oldRdd.unpersist(true)
>>
>
> According to my usage pattern i tried to don't unpersist the intermediate
> RDDs (i.e. oldRdd) but nothing change.
>
> Any hints? How could i debug this?
>
>
>
> 2014-04-14 12:55 GMT+02:00 Andrew Ash <and...@andrewash.com>:
>
> A lot of your time is being spent in garbage collection (second image).
>>  Maybe your dataset doesn't easily fit into memory?  Can you reduce the
>> number of new objects created in myFun?
>>
>> How big are your heap sizes?
>>
>> Another observation is that in the 4th image some of your RDDs are
>> massive and some are tiny.
>>
>>
>> On Mon, Apr 14, 2014 at 11:45 AM, Andrea Esposito <and1...@gmail.com>wrote:
>>
>>> Hi all,
>>>
>>> i'm developing an iterative computation over graphs but i'm struggling
>>> with some embarrassing low performaces.
>>>
>>> The computation is heavily iterative and i'm following this rdd usage
>>> pattern:
>>>
>>> newRdd = oldRdd.map(myFun).persist(myStorageLevel)
>>>>
>>> newRdd.foreach(x => {}) // Force evaluation
>>>> oldRdd.unpersist(true)
>>>>
>>>
>>> I'm using a machine equips by 30 cores and 120 GB of RAM.
>>> As an example i've run with a small graph of 4000 verts and 80 thousand
>>> edges and the performance at the first iterations are 10+ minutes and after
>>> they take lots more.
>>> I attach the Spark UI screenshots of just the first 2 iterations.
>>>
>>> I tried with MEMORY_ONLY_SER and MEMORY_AND_DISK_SER and also i changed
>>> the "spark.shuffle.memoryFraction" to 0.3 but nothing changed (with so lot
>>> of RAM for 4E10 verts these settings are quite pointless i guess).
>>>
>>> How should i continue to investigate?
>>>
>>> Any advices are very very welcome, thanks.
>>>
>>> Best,
>>> EA
>>>
>>
>>
>

Reply via email to