----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/23358/#review47895 -----------------------------------------------------------
Great work Yan. I know it takes a lot of work to make sense of what another framework is doing. I've added some questions about the details below. Some might be hard to answer, but I think they're worth thinking about. If you get stuck, an option would be to commit and publish a first version of this comparison, then ask the Spark folks for their feedback, and incorporate any corrections as needed. I'll leave that to you to judge. docs/learn/documentation/0.7.0/comparisons/spark-streaming.md <https://reviews.apache.org/r/23358/#comment84120> In order for Jekyll to recognise this as a bulleted list in Markdown, you need to leave a blank line before the first bullet point. At the moment it just renders as one long paragraph with some asterisks in the middle. docs/learn/documentation/0.7.0/comparisons/spark-streaming.md <https://reviews.apache.org/r/23358/#comment84128> 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. docs/learn/documentation/0.7.0/comparisons/spark-streaming.md <https://reviews.apache.org/r/23358/#comment84138> Does this state DStream provide any key-value access or other query model? If it's just a stream of records, that would imply that every time a batch of input records is processed, the stream processor also needs to consume the entire state DStream. That's fine if the state is small, but with a large amount of state (multiple GB), it would probably get very inefficient. If this is true, it would further support our "Samza is good if you have lots of state" story. Also: you don't mention anything about stream joins in this comparison. I see Spark has a join operator -- do you know what it does? Does it just take one batch from each input stream and join within those batches? Or can you do joins across a longer window, or against a table? Since joins typically involve large amounts of state, they are worth highlighting as an area where Samza may be stronger. "Everytime updateStateByKey is applied, you will get a new state DStream": presumably you get a new DStream once per batch, not for every single message within a batch? docs/learn/documentation/0.7.0/comparisons/spark-streaming.md <https://reviews.apache.org/r/23358/#comment84134> Typo: "intermedia" Is this periodic writing of state to HDFS basically a form of checkpointing? If so, it might be worth calling it such. The state management page of the docs also calls out that checkpointing the entire task state is inefficient if the state is large. Could do a cross-reference. docs/learn/documentation/0.7.0/comparisons/spark-streaming.md <https://reviews.apache.org/r/23358/#comment84140> "its" -> "it's" "should support" -> "supports" Might be worth pointing out that in Samza you can also plug in other storage engines (../container/state-management.html#other-storage-engines), which enables great flexibility in the stream processing algorithms you can use. docs/learn/documentation/0.7.0/comparisons/spark-streaming.md <https://reviews.apache.org/r/23358/#comment84145> "send them to executors" -> "sending them to executors" "the parallelism is simply accomplished by normal RDD operations, such as map, reduceByKey, reduceByWindow" -- this is unclear. How are these operations parallelized? I assume each RDD batch is processed sequentially by a single processing task, but different batches can be processed on different machines. Is that right? If your input stream is partitioned (i.e. there are multiple receivers), does each batch include only messages from a single partition, or are all the partitions combined into a single batch? Can you repartition a stream, e.g. when grouping on a field, to ensure that all messages with the same value in that field get grouped together? (akin to the shuffle phase in MapReduce) How does partitioning play into the ordering guarantees? Even if each partition is ordered, there typically isn't a deterministic ordering of messages across partitions; how does this interact with Spark's determinism requirement for operators? docs/learn/documentation/0.7.0/comparisons/spark-streaming.md <https://reviews.apache.org/r/23358/#comment84147> "batch processing" -> "batch processes" docs/learn/documentation/0.7.0/comparisons/spark-streaming.md <https://reviews.apache.org/r/23358/#comment84151> "a small difference": if Spark may lose messages and Samza guarantees delivery, I'd say that's a pretty big difference ;-) In fact, it's potentially a serious problem for Spark. Do you have a reference for this? Seems like the kind of problem which could be fixed, so perhaps they're working on it. docs/learn/documentation/0.7.0/comparisons/spark-streaming.md <https://reviews.apache.org/r/23358/#comment84152> "reprocess the same data from since data was processed" -- sentence seems to have too many words in it? docs/learn/documentation/0.7.0/comparisons/spark-streaming.md <https://reviews.apache.org/r/23358/#comment84173> Is this really true? I get the impression that with Spark Streaming you build an entire processing graph with a DSL API, and deploy that entire graph as one unit. The communication between the nodes in that graph (in the form of DStreams) is provided by the framework. In that way it looks similar to Storm. Samza is totally different -- each job is just a message-at-a-time processor, and there is no framework support for topologies. Output of a processing task always needs to go back to a message broker (e.g. Kafka). A positive consequence of Samza's design is that a job's output can be consumed by multiple unrelated jobs, potentially run by different teams, and those jobs are isolated from each other through Kafka's buffering. That is not the case with Storm's (and Spark Streaming's?) framework-internal streams. Although a Storm/Spark job could in principle write its output to a message broker, the framework doesn't really make this easy. It seems that Storm/Spark aren't intended to used in a way where one topology's output is another topology's input. By contrast, in Samza, that mode of usage is standard. docs/learn/documentation/0.7.0/comparisons/spark-streaming.md <https://reviews.apache.org/r/23358/#comment84174> Prefer "young" to "immature" perhaps? We shouldn't sell ourselves too short :) - Martin Kleppmann 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 > >
