> On July 16, 2014, 8:36 p.m., Martin Kleppmann wrote:
> > docs/learn/documentation/0.7.0/comparisons/spark-streaming.md, line 36
> > <https://reviews.apache.org/r/23358/diff/3/?file=632630#file632630line36>
> >
> >     This paragraph seems contradictory -- does Spark guarantee ordering or 
> > not? And what do you mean with "is not emphasized in the document"?
> >     
> >     My understanding is that Spark's transformation operators must be 
> > side-effect-free, so the order in which batches are processed is 
> > irrelevant. When one batch depends on the output of a previous batch (e.g. 
> > a window-based operation), Spark Streaming guarantees that the correct 
> > previous batch is used as input to the subsequent batch (which is 
> > effectively ordering, even if some of the execution may actually happen in 
> > parallel).
> >     
> >     I'm not sure about ordering of output operations (which may have 
> > side-effects).
> >     
> >     Another thing -- I believe Spark Streaming requires transformation 
> > operators to be deterministic. Is that true? If so, it would be worth 
> > mentioning, because 
> >     that may make it unsuitable for nondeterministic processing, e.g. a 
> > randomized machine learning algorithm. Samza has no such requirement.
> >     
> >     "Spark Streaming supports at-least once messaging semantics": you say 
> > below that Spark Streaming may lose messages if the receiver task fails. If 
> > this is the case, the guarantee is neither at-least-once nor at-most-once, 
> > but more like zero-or-more-times.
> 
> Yan Fang wrote:
>     When I say "is not emphasized in the document", mean that I could not 
> find relevant documents. From my test, the messages order in one DStream 
> seems guaranteed. But if you combine some DStreams in the process, no order 
> is guaranteed. 
>     
>     you are right, transformation operations are side-effect-free and output 
> operations (should) have the side-effects. And all transformation operations 
> only happen after output operations are called (because of lazy 
> implementation). 
>     
>     So I am a little conservative about the order of messages in Spark 
> Streaming in case I write something wrong.
>     
>     yes for "transformation operators to be deterministic". Because you only 
> apply the operations in a deterministic stream. Will mention that.
>     
>     they do lose data and work on that. 
> https://issues.apache.org/jira/browse/SPARK-1730, 
> https://issues.apache.org/jira/browse/SPARK-1647. It's a little weird in 
> Kafka situation. Because of the consumer offset, it does not lose data but 
> processes too many messages at the first, say , 2s, when you bring up the 
> receiver after the failure. Maybe I should mention it as well?
> 
> Martin Kleppmann wrote:
>     "not emphasized": maybe say that in Spark, since messages are processed 
> in batches by side-effect-free operators, the exact ordering of messages is 
> not important in Spark.
>     
>     Good find on the data loss issues, I'd suggest linking to SPARK-1647. I 
> don't understand the issue with Kafka. When it comes back after a failure, 
> does it start consuming from the latest offset, or some older offset?
> 
> Yan Fang wrote:
>     In terms of Kafka, when Spark Streaming restarts, it starts from the 
> older offset where it fails. That means, if Spark Streaming is using Kafka as 
> the input stream, it will not lose data in a receiver/driver failure 
> scenario. However, since there are many unprocessed messages in the Kafka ( 
> because it does not consume any data during the failure time), it will 
> consume all the unprocessed messages at the first interval. After that, it 
> goes to normal situation where it consumes as the same rate as the data is 
> coming. Now they have a patch 
> https://issues.apache.org/jira/browse/SPARK-1341 to control the rate.
>     
>     But for sure it loses data if it's using Flume/Twitter data as the input 
> stream.

With "older offset", do you mean the oldest offset (which might be data that is 
several weeks old)? Or from the last checkpoint?

If you're using an input system that doesn't buffer unprocessed messages, like 
Flume, Twitter or the IRC example in hello-samza, then Samza similarly loses 
data on container restart. There's probably no way around that. So it looks 
like Samza and Spark are actually the same in that regard?


- Martin


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/23358/#review47895
-----------------------------------------------------------


On July 15, 2014, 6:15 p.m., Yan Fang wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/23358/
> -----------------------------------------------------------
> 
> (Updated July 15, 2014, 6:15 p.m.)
> 
> 
> Review request for samza.
> 
> 
> Repository: samza
> 
> 
> Description
> -------
> 
> Comparison of Spark Streaming and Samza
> 
> 
> Diffs
> -----
> 
>   docs/learn/documentation/0.7.0/comparisons/spark-streaming.md PRE-CREATION 
>   docs/learn/documentation/0.7.0/comparisons/storm.md 4a21094 
>   docs/learn/documentation/0.7.0/index.html 149ff2b 
> 
> Diff: https://reviews.apache.org/r/23358/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Yan Fang
> 
>

Reply via email to