Adi, How far away are we from having something a prototype patch to play with? >
We are working to share a prototype next week. Though the code will evolve to match the APIs and design as it shapes up, but it will be great if people can take a look and provide feedback. Couple of observations: > - Since the input source for each processor is always Kafka, you get basic > client side partition management out of the box it use the high level > consumer. > That's right. The plan is to propose moving the partition assignment in the consumer to the client-side (proposal coming up soon) and then use that in Kafka streams and copycat. > - The KIP states that cmd line tools will be provided to deploy as a > separate service. Is the proposed scope limited to providing a library with > which makes it possible build stream-processing-as- a-service or provide > such a service within Kafka itself? I think the KIP might've been a little misleading on this point. The scope is to provide a library and have any integrations with other resource management frameworks (Slider for YARN and Marathon for Mesos) live outside Kafka. Having said that, in order to just get started with a simple stream processing example, you still need basic scripts to get going. Those are not anywhere similar in scope to what you'd expect in order to run this as a service. Thanks, Neha On Mon, Jul 27, 2015 at 10:57 PM, Neha Narkhede <n...@confluent.io> wrote: > Gwen, > > We have a compilation of notes from comparison with other systems. They > might be missing details that folks who worked on that system might be able > to point out. We can share that and discuss further on the KIP call. > > We do hope to include a DSL since that is the most natural way of > expressing stream processing operations on top of the processor client. The > DSL layer should be equivalent to that provided by Spark streaming or Flink > in terms of expressiveness though there will be differences in > implementation. Our client is intended to be simpler, with minimum external > dependencies since it integrates closely with Kafka. This is really what > most application development is hoping to get - a lightweight library on > top of Kafka that allows them to process streams of data. > > Thanks > Neha > > On Mon, Jul 27, 2015 at 8:20 PM, Gwen Shapira <gshap...@cloudera.com> > wrote: > >> Hi, >> >> Since we will be discussing KIP-28 in the call tomorrow, can you >> update the KIP with the feature-comparison with existing solutions? >> I admit that I do not see a need for single-event-producer-consumer >> pair (AKA Flume Interceptor). I've seen tons of people implement such >> apps in the last year, and it seemed easy. Now, perhaps we were doing >> it all wrong... but I'd like to know how :) >> >> If we are talking about a bigger story (i.e. DSL, real >> stream-processing, etc), thats a different discussion. I've seen a >> bunch of misconceptions about SparkStreaming in this discussion, and I >> have some thoughts in that regard, but I'd rather not go into that if >> thats outside the scope of this KIP. >> >> Gwen >> >> >> On Fri, Jul 24, 2015 at 9:48 AM, Guozhang Wang <wangg...@gmail.com> >> wrote: >> > Hi Ewen, >> > >> > Replies inlined. >> > >> > On Thu, Jul 23, 2015 at 10:25 PM, Ewen Cheslack-Postava < >> e...@confluent.io> >> >> > wrote: >> > >> >> Just some notes on the KIP doc itself: >> >> >> >> * It'd be useful to clarify at what point the plain consumer + custom >> code >> >> + producer breaks down. I think trivial filtering and aggregation on a >> >> single stream usually work fine with this model. Anything where you >> need >> >> more complex joins, windowing, etc. are where it breaks down. I think >> most >> >> interesting applications require that functionality, but it's helpful >> to >> >> make this really clear in the motivation -- right now, Kafka only >> provides >> >> the lowest level plumbing for stream processing applications, so most >> >> interesting apps require very heavyweight frameworks. >> >> >> > >> > I think for users to efficiently express complex logic like joins >> > windowing, etc, a higher-level programming interface beyond the >> process() >> > interface would definitely be better, but that does not necessarily >> require >> > a "heavyweight" frameworks, which usually includes more than just the >> > high-level functional programming model. I would argue that an >> alternative >> > solution would better be provided for users who want some high-level >> > programming interface but not a heavyweight stream processing framework >> > that include the processor library plus another DSL layer on top of it. >> > >> > >> > >> >> * I think the feature comparison of plain producer/consumer, stream >> >> processing frameworks, and this new library is a good start, but we >> might >> >> want something more thorough and structured, like a feature matrix. >> Right >> >> now it's hard to figure out exactly how they relate to each other. >> >> >> > >> > Cool, I can do that. >> > >> > >> >> * I'd personally push the library vs. framework story very strongly -- >> the >> >> total buy-in and weak integration story of stream processing >> frameworks is >> >> a big downside and makes a library a really compelling (and currently >> >> unavailable, as far as I am aware) alternative. >> >> >> > >> > Are you suggesting there are still some content missing about the >> > motivations of adding the proposed library in the wiki page? >> > >> > >> >> * Comment about in-memory storage of other frameworks is interesting >> -- it >> >> is specific to the framework, but is supposed to also give performance >> >> benefits. The high-level functional processing interface would allow >> for >> >> combining multiple operations when there's no shuffle, but when there >> is a >> >> shuffle, we'll always be writing to Kafka, right? Spark (and presumably >> >> spark streaming) is supposed to get a big win by handling shuffles such >> >> that the data just stays in cache and never actually hits disk, or at >> least >> >> hits disk in the background. Will we take a hit because we always >> write to >> >> Kafka? >> >> >> > >> > I agree with Neha's comments here. One more point I want to make is >> > materializing to Kafka is not necessarily much worse than keeping data >> in >> > memory if the downstream consumption is caught up such that most of the >> > reads will be hitting file cache. I remember Samza has illustrated that >> > under such scenarios its throughput is actually quite comparable to >> Spark >> > Streaming / Storm. >> > >> > >> >> * I really struggled with the structure of the KIP template with >> Copycat >> >> because the flow doesn't work well for proposals like this. They >> aren't as >> >> concrete changes as the KIP template was designed for. I'd completely >> >> ignore that template in favor of optimizing for clarity if I were you. >> >> >> >> -Ewen >> >> >> >> On Thu, Jul 23, 2015 at 5:59 PM, Guozhang Wang <wangg...@gmail.com> >> wrote: >> >> >> >> > Hi all, >> >> > >> >> > I just posted KIP-28: Add a transform client for data processing >> >> > < >> >> > >> >> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-28+-+Add+a+transform+client+for+data+processing >> >> > > >> >> > . >> >> > >> >> > The wiki page does not yet have the full design / implementation >> details, >> >> > and this email is to kick-off the conversation on whether we should >> add >> >> > this new client with the described motivations, and if yes what >> features >> >> / >> >> > functionalities should be included. >> >> > >> >> > Looking forward to your feedback! >> >> > >> >> > -- Guozhang >> >> > >> >> >> >> >> >> >> >> -- >> >> Thanks, >> >> Ewen >> >> >> > >> > >> > >> > -- >> > -- Guozhang >> > > > > -- > Thanks, > Neha > -- Thanks, Neha