Re: *ByKey aggregations: performance + order

2015-01-15 Thread Sean Owen
I'm interested too and don't know for sure but I do not think this case is optimized this way. However if you know your keys aren't split across partitions and you have small enough partitions you can implement the same grouping with mapPartitions and Scala. On Jan 15, 2015 1:27 AM, Tobias

Re: *ByKey aggregations: performance + order

2015-01-14 Thread Tobias Pfeiffer
Sean, thanks for your message. On Wed, Jan 14, 2015 at 8:36 PM, Sean Owen so...@cloudera.com wrote: On Wed, Jan 14, 2015 at 4:53 AM, Tobias Pfeiffer t...@preferred.jp wrote: OK, it seems like even on a local machine (with no network overhead), the groupByKey version is about 5 times slower

Re: *ByKey aggregations: performance + order

2015-01-14 Thread Sean Owen
On Wed, Jan 14, 2015 at 4:53 AM, Tobias Pfeiffer t...@preferred.jp wrote: Now I don't know (yet) if all of the functions I want to compute can be expressed in this way and I was wondering about *how much* more expensive we are talking about. OK, it seems like even on a local machine (with no

Re: *ByKey aggregations: performance + order

2015-01-13 Thread Tobias Pfeiffer
Hi, On Wed, Jan 14, 2015 at 12:11 PM, Tobias Pfeiffer t...@preferred.jp wrote: Now I don't know (yet) if all of the functions I want to compute can be expressed in this way and I was wondering about *how much* more expensive we are talking about. OK, it seems like even on a local machine

*ByKey aggregations: performance + order

2015-01-13 Thread Tobias Pfeiffer
Hi, I have an RDD[(Long, MyData)] where I want to compute various functions on lists of MyData items with the same key (this will in general be a rather short lists, around 10 items per key). Naturally I was thinking of groupByKey() but was a bit intimidated by the warning: This operation may be