-- Hi Jay, Thanks for forwarding this.
-- Hi TD, Thanks for pointing this out. That overview was written about one year ago. It is out-of-date. Good to get some critiques from your side. Will update it soon. Created SAMZA-698 <https://issues.apache.org/jira/browse/SAMZA-698> to track this. Thank you. Cheers, Fang, Yan [email protected] On Tue, Jun 2, 2015 at 9:40 AM, Jay Kreps <[email protected]> wrote: > Hey guys, > > Here were some critiques of our system comparison page from Tathagata at > Databricks. > > -Jay > > ---------- Forwarded message ---------- > From: Tathagata Das <[email protected]> > Date: Thu, May 14, 2015 at 1:15 PM > Subject: About Spark Streaming overview in Samza docs > To: Jay Kreps <[email protected]> > > > Hello Jay, > > I am not sure if you remember me from our earlier (a year or so) phone > conversation along with Patrick Wendell, so let me introduce myself. I am > Tathagata Das (aka TD), and I am the technical lead behind Spark Streaming. > We had chatted earlier about various topics related to Kafka and I hope we > can chat more about it some time soon. > > However, in this mail, I wanted to talk a bit about Samza's description of > Spark Streaming > < > http://samza.apache.org/learn/documentation/0.9/comparisons/spark-streaming.html > >. > Though I sort of assumed that you are the right person to talk. But that > isnt the case, feel free to redirect me to whoever you think is the best > person for this. > > The overview of Spark Streaming is pretty good! I myself would not have > been able to put the high-level architecture of Spark Streaming so > succinctly. That said, there are a few pieces of information that are > outdated and it will be good to update the page to avoid confusion. Here > are some of them. > > 1.* " Spark Streaming does not gurantee at-least-once or at-most-once > messaging semantics"* - This is outdated information. In Spark 1.2, we > introduced write ahead logs > < > https://databricks.com/blog/2015/01/15/improved-driver-fault-tolerance-and-zero-data-loss-in-spark-streaming.html > > > that can guarantee at least once processing for any reliable source, > despite driver and worker failures. In addition, in Spark 1.3 we > introduced a > new way > < > https://databricks.com/blog/2015/03/30/improvements-to-kafka-integration-of-spark-streaming.html > > > to process data from Kafka, such that it achieves end-to-end exactly-once > processing if data store updates are idempotent or transactional (BTW, did > I say Kafka is *amazing* which allowed us to do this crazy new approach?). > > 2. *"Spark Streaming may lose data if the failure happens when the data is > received but not yet replicated to other nodes (also see SPARK-1647)"* - > Again, this has changed in between Spark 1.1 - 1.3. For Flume, we added > Flume polling stream that uses Flume transactions to guarantee that data is > properly replicated or retransmitted on receiver failure. Driver failures > handled by write ahead logs. For Kafka, the new approach does not even need > replication as it treats Kafka like a file system, reading segments of log > as needed. > > 3. *"it is unsuitable for nondeterministic processing, e.g. a randomized > machine learning algorithm"* - It is incorrect to say that Spark Streaming > is unsuitable. We suggest using deterministic operations only to ensure > that the developers always get the expected results even if there are > failures. Just like MapReduce, there is nothing stopping any user from > implementing a non-determinstic algorithm on Spark Streaming, as long as > the user is aware of its consequence of fault-tolerance guarantees (results > may change due to failures). Furthermore, randomized streaming machine > learning algorithms can still be implemented using deterministic > transformations (using pseudo random numbers, etc.). There are quite a few > random sampling (e.g. RDD.sample() > < > https://spark.apache.org/docs/latest/api/scala/index.html#org.apache.spark.rdd.RDD > >) > and randomized algorithms > < > https://databricks.com/blog/2015/01/21/random-forests-and-boosting-in-mllib.html > > > in core Spark and MLlib (Spark's machine learning library), and the same > techniques can be used to implement "deterministic" randomized machine > learning algorithms on Spark Streaming. > > 4. *"When a driver node fails in Spark Streaming, Spark’s standalone > cluster mode will restart the driver node automatically. But it is > currently not supported in YARN and Mesos."* - YARN supports automatically > restarting the AM, global default being at most 1 restart ( > yarn.resourcemanager.am.max-attempts > < > http://hadoop.apache.org/docs/r2.7.0/hadoop-yarn/hadoop-yarn-common/yarn-default.xml > >). > On Mesos, applications are often launched using Marathon > <https://mesosphere.github.io/marathon/docs/>, which also supports > restarting. > > 5. *"Samza is still young, but has just released version 0.7.0."* - > Incorrect ;) > > Sorry for this long post. I am happy to get on phone/hangout/skype with you > if more clarifications are needed. And independent of all this, feel free > to email me about anything anytime. > > Thanks! > > TD >
