Re: Understanding RDD.GroupBy OutOfMemory Exceptions

2015-01-07 Thread asimjalis
Hi Patrick: Do you know what the status of this issue is? Is there a JIRA that is tracking this issue? Thanks. Asim Patrick Wendell writes: "Within a partition things will spill - so the current documentation is correct. This spilling can only occur *across keys* at the moment. Spilling cannot

Re: Understanding RDD.GroupBy OutOfMemory Exceptions

2014-08-25 Thread Patrick Wendell
Hey Andrew, We might create a new JIRA for it, but it doesn't exist yet. We'll create JIRA's for the major 1.2 issues at the beginning of September. - Patrick On Mon, Aug 25, 2014 at 8:53 AM, Andrew Ash wrote: > Hi Patrick, > > For the spilling within on key work you mention might land in Spa

Re: Understanding RDD.GroupBy OutOfMemory Exceptions

2014-08-25 Thread Andrew Ash
Hi Patrick, For the spilling within on key work you mention might land in Spark 1.2, is that being tracked in https://issues.apache.org/jira/browse/SPARK-1823 or is there another ticket I should be following? Thanks! Andrew On Tue, Aug 5, 2014 at 3:39 PM, Patrick Wendell wrote: > Hi Jens, > >

Re: Understanding RDD.GroupBy OutOfMemory Exceptions

2014-08-05 Thread Patrick Wendell
Hi Jens, Within a partition things will spill - so the current documentation is correct. This spilling can only occur *across keys* at the moment. Spilling cannot occur within a key at present. This is discussed in the video here: https://www.youtube.com/watch?v=dmL0N3qfSc8&index=3&list=PL-x35fyl

Re: Understanding RDD.GroupBy OutOfMemory Exceptions

2014-08-05 Thread Jens Kristian Geyti
Patrick Wendell wrote > In the latest version of Spark we've added documentation to make this > distinction more clear to users: > https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala#L390 That is a very good addition to the documentation. Nic

Re: Understanding RDD.GroupBy OutOfMemory Exceptions

2014-08-05 Thread Patrick Wendell
this, or even trying to understand what's going on - for > the same reason, I'm not looking for a solution to my specific problem, as > much as I'm looking for insight into how to reliably group datasets in > Spark. > > Notice that I've also posted this ques

Understanding RDD.GroupBy OutOfMemory Exceptions

2014-08-05 Thread Jens Kristian Geyti
I'm doing a simple groupBy on a fairly small dataset (80 files in HDFS, few gigs in total, line based, 500-2000 chars per line). I'm running Spark on 8 low-memory machines in a yarn cluster, i.e. something along the lines of: spark-submit ... --master yarn-client --num-executors 8 --executor-me