Re: Transforming the Dstream vs transforming each RDDs in the Dstream.

2014-10-29 Thread Gerard Maas
Hi TD, Thanks a lot for the comprehensive answer. I think this explanation deserves some place in the Spark Streaming tuning guide. -kr, Gerard. On Thu, Oct 23, 2014 at 11:41 PM, Tathagata Das tathagata.das1...@gmail.com wrote: Hey Gerard, This is a very good question! *TL;DR: *The

Re: Transforming the Dstream vs transforming each RDDs in the Dstream.

2014-10-29 Thread Tathagata Das
Good idea, will do for 1.2 release. On Oct 29, 2014 9:50 AM, Gerard Maas gerard.m...@gmail.com wrote: Hi TD, Thanks a lot for the comprehensive answer. I think this explanation deserves some place in the Spark Streaming tuning guide. -kr, Gerard. On Thu, Oct 23, 2014 at 11:41 PM,

Re: Transforming the Dstream vs transforming each RDDs in the Dstream.

2014-10-29 Thread jay vyas
Hi tathagata. I actually had a few minor improvements to spark streaming in SPARK-4040. possibly i could weave this in w/ my pr ? On Wed, Oct 29, 2014 at 1:59 PM, Tathagata Das tathagata.das1...@gmail.com wrote: Good idea, will do for 1.2 release. On Oct 29, 2014 9:50 AM, Gerard Maas

Re: Transforming the Dstream vs transforming each RDDs in the Dstream.

2014-10-23 Thread Tathagata Das
Hey Gerard, This is a very good question! *TL;DR: *The performance should be same, except in case of shuffle-based operations where the number of reducers is not explicitly specified. Let me answer in more detail by dividing the set of DStream operations into three categories. *1. Map-like

Re: Transforming the Dstream vs transforming each RDDs in the Dstream.

2014-10-22 Thread Gerard Maas
Thanks Matt, Unlike the feared RDD operations on the driver, it's my understanding that these Dstream ops on the driver are merely creating an execution plan for each RDD. My question still remains: Is it better to foreachRDD early in the process or do as much Dstream transformations before going

Re: Transforming the Dstream vs transforming each RDDs in the Dstream.

2014-10-22 Thread Gerard Maas
PS: Just to clarify my statement: Unlike the feared RDD operations on the driver, it's my understanding that these Dstream ops on the driver are merely creating an execution plan for each RDD. With feared RDD operations on the driver I meant to contrast an rdd action like rdd.collect that would

Re: Transforming the Dstream vs transforming each RDDs in the Dstream.

2014-10-20 Thread Gerard Maas
Pinging TD -- I'm sure you know :-) -kr, Gerard. On Fri, Oct 17, 2014 at 11:20 PM, Gerard Maas gerard.m...@gmail.com wrote: Hi, We have been implementing several Spark Streaming jobs that are basically processing data and inserting it into Cassandra, sorting it among different keyspaces.

Re: Transforming the Dstream vs transforming each RDDs in the Dstream.

2014-10-20 Thread Matt Narrell
http://spark.apache.org/docs/latest/streaming-programming-guide.html http://spark.apache.org/docs/latest/streaming-programming-guide.html foreachRDD is executed on the driver…. mn On Oct 20, 2014, at 3:07 AM, Gerard Maas gerard.m...@gmail.com wrote: Pinging TD -- I'm sure you know :-)

Transforming the Dstream vs transforming each RDDs in the Dstream.

2014-10-17 Thread Gerard Maas
Hi, We have been implementing several Spark Streaming jobs that are basically processing data and inserting it into Cassandra, sorting it among different keyspaces. We've been following the pattern: dstream.foreachRDD(rdd = val records = rdd.map(elem = record(elem))