Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-28 Thread Matthias J. Sax
> -Jay >>> >>> On Wed, Mar 22, 2017 at 11:25 AM, Guozhang Wang wrote: >>> >>>> Regarding the naming of `StreamsTopologyBuilder` v.s. `StreamsBuilder` that >>>> are going to be used in DSL, I agree both has their arguments: >>>> &

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-27 Thread Matthias J. Sax
e using the DSL layer probably do not need to be aware >>> (or rather, "learn about") of the "topology" concept, although this concept >>> is a publicly exposed one in Kafka Streams. >>> >>> 2. On the other side, StreamsBuilder#build() returning a

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-23 Thread Matthias J. Sax
uilder#build() returning a Topology object >> sounds a little weird, at least to me (admittedly subjective matter). >> >> >> Since the second bullet point seems to be more "subjective" and many people >> are not worried about it, I'm OK to go with the other opt

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-22 Thread Jay Kreps
and many people > are not worried about it, I'm OK to go with the other option. > > > Guozhang > > > On Wed, Mar 22, 2017 at 8:58 AM, Michael Noll > wrote: > > > Forwarding to kafka-user. > > > > > > -- Forwarded message --

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-22 Thread Guozhang Wang
worried about it, I'm OK to go with the other option. Guozhang On Wed, Mar 22, 2017 at 8:58 AM, Michael Noll wrote: > Forwarding to kafka-user. > > > -- Forwarded message -- > From: Michael Noll > Date: Wed, Mar 22, 2017 at 8:48 AM > Subject: Re: [DIS

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-22 Thread Michael Noll
Forwarding to kafka-user. -- Forwarded message -- From: Michael Noll Date: Wed, Mar 22, 2017 at 8:48 AM Subject: Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API To: d...@kafka.apache.org Matthias, > @Michael: > > You seemed to agree with Jay about not exp

Re: Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-21 Thread Guozhang Wang
ns about future changes, but > the very first concept we explain in the Kafka Streams documentation is > "Stream Processing Topology": > https://kafka.apache.org/0102/documentation/streams#streams_concepts > > -Michael > > > > On Mon, Mar 20, 2017 at 7:55 PM, Matth

Re: Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-20 Thread Michael Noll
umentation/streams#streams_concepts -Michael On Mon, Mar 20, 2017 at 7:55 PM, Matthias J. Sax wrote: > \cc users list > > > -------- Forwarded Message ---- > Subject: Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API > Date: Mon, 20 Mar 2017 11:51:01 -0700

Fwd: Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-20 Thread Matthias J. Sax
\cc users list Forwarded Message Subject: Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API Date: Mon, 20 Mar 2017 11:51:01 -0700 From: Matthias J. Sax Organization: Confluent Inc To: d...@kafka.apache.org I want to push this discussion further. Guozhang's arg

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-14 Thread Guozhang Wang
I'd like to keep the term "Topology" inside the builder class since, as Matthias mentioned, this builder#build() function returns a "Topology" object, whose type is a public class anyways. Although you can argue to let users always call "new KafkaStreams(builder.build())" I think it is still more

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-14 Thread Matthias J. Sax
Thanks for your input Michael. >> - KafkaStreams as the new name for the builder that creates the logical >> plan, with e.g. `KafkaStreams.stream("intput-topic")` and >> `KafkaStreams.table("input-topic")`. I don't thinks this is a good idea, for multiple reasons: (1) We would reuse a name for a

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-14 Thread Michael Noll
I see Jay's point, and I agree with much of it -- notably about being careful which concepts we do and do not expose, depending on which user group / user type is affected. That said, I'm not sure yet whether or not we should get rid of "Topology" (or a similar term) in the DSL. For what it's wor

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-13 Thread Sriram Subramanian
StreamsBuilder would be my vote. > On Mar 13, 2017, at 9:42 PM, Jay Kreps wrote: > > Hey Matthias, > > Make sense, I'm more advocating for removing the word topology than any > particular new replacement. > > -Jay > > On Mon, Mar 13, 2017 at 12:30 PM, Matthias J. Sax > wrote: > >> Jay, >>

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-13 Thread Jay Kreps
Hey Matthias, Make sense, I'm more advocating for removing the word topology than any particular new replacement. -Jay On Mon, Mar 13, 2017 at 12:30 PM, Matthias J. Sax wrote: > Jay, > > thanks for your feedback > > > What if instead we called it KStreamsBuilder? > > That's the current name an

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-13 Thread Matthias J. Sax
Steven, thanks for your feedback. I am not sure about KafkaStreamsBuilder (even if agree that it is better than KStreamBuilder), because it sounds like a builder that creates a KafkaStreams instance. But that's of course not the case. It builds a Topology -- that was the reason to consider callin

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-13 Thread Steven Schlansker
> On Mar 13, 2017, at 12:30 PM, Matthias J. Sax wrote: > > Jay, > > thanks for your feedback > >> What if instead we called it KStreamsBuilder? > > That's the current name and I personally think it's not the best one. > The main reason why I don't like KStreamsBuilder is, that we have the > c

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-13 Thread Matthias J. Sax
Jay, thanks for your feedback > What if instead we called it KStreamsBuilder? That's the current name and I personally think it's not the best one. The main reason why I don't like KStreamsBuilder is, that we have the concepts of KStreams and KTables, and the builder creates both. However, the n

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-13 Thread Jay Kreps
Two things: 1. This is a minor thing but the proposed new name for KStreamBuilder is StreamsTopologyBuilder. I actually think we should not put topology in the name as topology is not a concept you need to understand at the kstreams layer right now. I'd think of three categories of con

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-11 Thread Matthias J. Sax
Thanks for your feedback Guozhang. 1) There are multiple ways to do this. Let me know what you think about all options: (a) I updated the KIP to this: > public final class Source implements Node { > public final String name; > // topicNames and topicPattern are mutually excl

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-10 Thread Guozhang Wang
One more question here: 3. with TopologyDescription, do we still want to keep the `KafkaStream.toString()` function? I think it may still have some advantage such that it contains tasks information after `KafkaStream#start()` has been called, but much of it is duplicate with the TopologyDescriptio

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-09 Thread Guozhang Wang
Thanks Matthias, the updated KIP lgtm overall. A couple of minor comments: 1. With regard to this class: public final class Source implements Node { public final String name; public final String topic; // can be topic name or pattern (as String) } Note that the source nod

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-09 Thread Michael Noll
Thanks for the update, Matthias. +1 to the points 1,2,3,4 you mentioned. Naming is always a tricky subject, but renaming KStreamBuilder to StreamsTopologyBuilder looks ok to me (I would have had a slight preference towards DslTopologyBuilder, but hey.) The most important aspect is, IMHO, what yo

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-08 Thread Matthias J. Sax
Hi, sorry for not replying earlier and thanks for all your feedback. After some more discussions I updated the KIP. The new proposal puts some other design considerations into account, that I want to highlight shortly. Those considerations, automatically resolve the concerns raised. First some an

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-15 Thread Mathieu Fenniak
On Wed, Feb 15, 2017 at 5:04 PM, Matthias J. Sax wrote: > - We also removed method #topologyBuilder() from KStreamBuilder because > we think #transform() should provide all functionality you need to > mix-an-match Processor API and DSL. If there is any further concern > about this, please let us

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-15 Thread Matthias J. Sax
Hi, according to the feedback, I updated the KIP, and limited its scope to some extend: - Instead of changing the creation of KafkaStreams instances, we keep the current pattern (we might do a follow up KIP on this though). - We also added a new method #describe() that returns a TopologyDescript

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-14 Thread Matthias J. Sax
You can already output any number of record within .transform() using the provided Context object from init()... -Matthias On 2/14/17 9:16 AM, Guozhang Wang wrote: >> and you can't output multiple records or branching logic from a > transform(); > > For output multiple records in transform, we

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-14 Thread Guozhang Wang
> and you can't output multiple records or branching logic from a transform(); For output multiple records in transform, we are currently working on https://issues.apache.org/jira/browse/KAFKA-4217, I think that should cover this use case. For branching the output in transform, I agree this is no

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-14 Thread Mathieu Fenniak
On Tue, Feb 14, 2017 at 9:37 AM, Damian Guy wrote: > > And about printing the topology for debuggability: I agrees this is a > > > potential drawback, and I'd suggest maintain some functionality to > build > > a > > > "dry topology" as Mathieu suggested; the difficulty is that, internally > > we

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-14 Thread Damian Guy
> And about printing the topology for debuggability: I agrees this is a > > potential drawback, and I'd suggest maintain some functionality to build > a > > "dry topology" as Mathieu suggested; the difficulty is that, internally > we > > need a different "copy" of the topology for each thread so th

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-14 Thread Mathieu Fenniak
On Tue, Feb 14, 2017 at 1:14 AM, Guozhang Wang wrote: > Some thoughts on the mixture usage of DSL / PAPI: > > There were some suggestions on mixing the usage of DSL and PAPI: > https://issues.apache.org/jira/browse/KAFKA-3455, and after thinking it a > bit more carefully, I'd rather not recommend

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-14 Thread Guozhang Wang
Some thoughts on the mixture usage of DSL / PAPI: There were some suggestions on mixing the usage of DSL and PAPI: https://issues.apache.org/jira/browse/KAFKA-3455, and after thinking it a bit more carefully, I'd rather not recommend users following this pattern, since in DSL this can always be ac

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-07 Thread Mathieu Fenniak
On Mon, Feb 6, 2017 at 2:35 PM, Matthias J. Sax wrote: > - adding KStreamBuilder#topologyBuilder() seems like be a good idea to > address any concern with limited access to TopologyBuilder and DSL/PAPI > mix-and-match approach. However, we should try to cover as much as > possible with #process()

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-06 Thread Gwen Shapira
Sounds good :) On Mon, Feb 6, 2017 at 5:40 PM, Matthias J. Sax wrote: > Gwen, > > thanks for your feedback. > > I completely agree that KStreamBuilder#merge() is miss placed and should > belong to KStream. However, I wanted to keep this KIP focus on one thing. > > As mentioned in a previous reply

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-06 Thread Matthias J. Sax
Gwen, thanks for your feedback. I completely agree that KStreamBuilder#merge() is miss placed and should belong to KStream. However, I wanted to keep this KIP focus on one thing. As mentioned in a previous reply, we plan to have at least one more KIP to clean up DSL -- this future KIP should inc

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-06 Thread Gwen Shapira
I like the cleanup a lot :) The cleaner lines between PAPI and DSL are very helpful to beginners who try to make sense of a long list of methods. I noticed that the "merge" method is still part of StreamBuilder. I thought it belongs inside KStream. Merge seems a lot like the SQL "union" operator,

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-06 Thread Matthias J. Sax
Mathieu, Damian, thanks a lot for your feedback. It's very valuable to see what, how and why people are using certain methods right now. We don't want to lock people out (that's why we put this KIP on users list, too) and we want to keep the ability to mix-and-match DSL and Processor API. Furthe

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-06 Thread Damian Guy
Hi Matthias, Thanks for the KIP. Should TopologyBuilder#build() and KStreamBuilder#build accept a StreamsConfig as an argument? Should we add a KStreamBuilder#topologyBuilder() for cases where people want to mix and match DSL & PAPI? Or do you think we already provide enough support for that via

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-06 Thread Mathieu Fenniak
Hi Matthias, I use a few of the methods that you're pointing out that will be deprecated and don't have an apparent alternative, so I wanted to just let you know what they are and what my use-cases are for them. First of all, I use a combination of DSL and PAPI in the same application very happil

Fwd: Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-04 Thread Matthias J. Sax
cc'ed from dev Forwarded Message Subject: Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API Date: Sat, 4 Feb 2017 11:30:46 -0800 From: Matthias J. Sax Organization: Confluent Inc To: d...@kafka.apache.org I think the right pattern should be to use TopologyBuilder

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-03 Thread radai
why is it so important to make those classes final ? On Fri, Feb 3, 2017 at 3:33 PM, Matthias J. Sax wrote: > Hi All, > > I did prepare a KIP to do some cleanup some of Kafka's Streaming API. > > Please have a look here: > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 120%3A+Cleanup+K

[DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-03 Thread Matthias J. Sax
Hi All, I did prepare a KIP to do some cleanup some of Kafka's Streaming API. Please have a look here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-120%3A+Cleanup+Kafka+Streams+builder+API Looking forward to your feedback! -Matthias signature.asc Description: OpenPGP digital signat