Re: RDD resiliency -- does it keep state?

2015-03-28 Thread Steve Loughran
It's worth adding that there's no guaranteed that re-evaluated work would be on the same host as before, and in the case of node failure, it is not guaranteed to be elsewhere. this means things that depend on host-local information is going to generate different numbers even if there are no

Re: RDD resiliency -- does it keep state?

2015-03-28 Thread Aaron Davidson
Note that speculation is off by default to avoid these kinds of unexpected issues. On Sat, Mar 28, 2015 at 6:21 AM, Steve Loughran ste...@hortonworks.com wrote: It's worth adding that there's no guaranteed that re-evaluated work would be on the same host as before, and in the case of node

Re: RDD resiliency -- does it keep state?

2015-03-28 Thread Michal Klos
got it thanks. Making sure everything is idempotent is definitely a critical piece for peace of mind. On Sat, Mar 28, 2015 at 1:47 PM, Aaron Davidson ilike...@gmail.com wrote: Note that speculation is off by default to avoid these kinds of unexpected issues. On Sat, Mar 28, 2015 at 6:21 AM,

RDD resiliency -- does it keep state?

2015-03-27 Thread Michal Klos
Hi Spark group, We haven't been able to find clear descriptions of how Spark handles the resiliency of RDDs in relationship to executing actions with side-effects. If you do an `rdd.foreach(someSideEffect)`, then you are doing a side-effect for each element in the RDD. If a partition goes down --

Re: RDD resiliency -- does it keep state?

2015-03-27 Thread Patrick Wendell
If you invoke this, you will get at-least-once semantics on failure. For instance, if a machine dies in the middle of executing the foreach for a single partition, that will be re-executed on another machine. It could even fully complete on one machine, but the machine dies immediately before