Re: Spark Streaming and message ordering

2015-02-20 Thread Cody Koeninger
For a given batch, for a given partition, the messages will be processed in order by the executor that is running that partition. That's because messages for the given offset range are pulled by the executor, not pushed from some other receiver. If you have speculative execution, yes, another

Re: Spark Streaming and message ordering

2015-02-20 Thread Jörn Franke
You may think as well if your use case really needs a very strict order, because configuring spark that it supports such a strict order means rendering most of benefits useless (failure handling, parallelism etc.). Usually, in a distributed setting you can order events, but this also means that

Re: Spark Streaming and message ordering

2015-02-20 Thread Neelesh
Thanks Jorn. Indeed, we do not need global ordering, since our data is partitioned well. We do not need ordering based on wallclock time, that would require waiting indefinitely. All we need is the execution of batches (not job submission) to happen in the same order they are generated, which

Re: Spark Streaming and message ordering

2015-02-20 Thread Neelesh
Thanks for the detailed response Cody. Our use case is to do some external lookups (cached and all) for every event, match the event against the looked up data, decide whether to write an entry in mysql and write it in the order in which the events arrived within a kafka partition. We don't need

Re: Spark Streaming and message ordering

2015-02-20 Thread Cody Koeninger
There is typically some slack between when a batch finishes executing and when the next batch is scheduled. You should be able to arrange your batch sizes / cluster resources to ensure that. If there isn't slack, your overall delay is going to keep increasing indefinitely. If you're inserting

Re: Spark Streaming and message ordering

2015-02-19 Thread Cody Koeninger
Kafka ordering is guaranteed on a per-partition basis. The high-level consumer api as used by the spark kafka streams prior to 1.3 will consume from multiple kafka partitions, thus not giving any ordering guarantees. The experimental direct stream in 1.3 uses the simple consumer api, and there

Re: Spark Streaming and message ordering

2015-02-19 Thread Neelesh
I had a chance to talk to TD today at the Strata+Hadoop Conf in San Jose. We talked a bit about this after his presentation about this - the short answer is spark streaming does not guarantee any sort of ordering (within batches, across batches). One would have to use updateStateByKey to collect

Re: Spark Streaming and message ordering

2015-02-19 Thread Neelesh
Even with the new direct streams in 1.3, isn't it the case that the job *scheduling* follows the partition order, rather than job *execution*? Or is it the case that the stream listens to job completion event (using a streamlistener) before scheduling the next batch? To compare with storm from a

Re: Spark Streaming and message ordering

2015-02-18 Thread jay vyas
This is a *fantastic* question. The idea of how we identify individual things in multiple DStreams is worth looking at. The reason being, that you can then fine tune your streaming job, based on the RDD identifiers (i.e. are the timestamps from the producer correlating closely to the order in

Spark Streaming and message ordering

2015-02-18 Thread Neelesh
There does not seem to be a definitive answer on this. Every time I google for message ordering,the only relevant thing that comes up is this - http://samza.apache.org/learn/documentation/0.8/comparisons/spark-streaming.html . With a kafka receiver that pulls data from a single kafka partition